001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.util; 019 020import java.io.IOException; 021import java.util.Collection; 022import java.util.EnumSet; 023import java.util.List; 024import java.util.Random; 025import java.util.concurrent.ThreadLocalRandom; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.fs.Path; 028import org.apache.hadoop.hbase.ClusterMetrics.Option; 029import org.apache.hadoop.hbase.MetaTableAccessor; 030import org.apache.hadoop.hbase.ServerName; 031import org.apache.hadoop.hbase.TableName; 032import org.apache.hadoop.hbase.ZooKeeperConnectionException; 033import org.apache.hadoop.hbase.client.Admin; 034import org.apache.hadoop.hbase.client.ClusterConnection; 035import org.apache.hadoop.hbase.client.Connection; 036import org.apache.hadoop.hbase.client.ConnectionFactory; 037import org.apache.hadoop.hbase.client.Put; 038import org.apache.hadoop.hbase.client.RegionInfo; 039import org.apache.hadoop.hbase.client.Table; 040import org.apache.hadoop.hbase.client.TableDescriptor; 041import org.apache.hadoop.hbase.master.RegionState; 042import org.apache.hadoop.hbase.master.ServerManager; 043import org.apache.hadoop.hbase.regionserver.HRegion; 044import org.apache.yetus.audience.InterfaceAudience; 045import org.apache.zookeeper.KeeperException; 046import org.slf4j.Logger; 047import org.slf4j.LoggerFactory; 048 049/** 050 * This class contains helper methods that repair parts of hbase's filesystem contents. 051 */ 052@InterfaceAudience.Private 053public class HBaseFsckRepair { 054 private static final Logger LOG = LoggerFactory.getLogger(HBaseFsckRepair.class); 055 056 /** 057 * Fix multiple assignment by doing silent closes on each RS hosting the region and then force ZK 058 * unassigned node to OFFLINE to trigger assignment by master. 059 * @param connection HBase connection to the cluster 060 * @param region Region to undeploy 061 * @param servers list of Servers to undeploy from 062 */ 063 public static void fixMultiAssignment(Connection connection, RegionInfo region, 064 List<ServerName> servers) throws IOException, KeeperException, InterruptedException { 065 // Close region on the servers silently 066 for (ServerName server : servers) { 067 closeRegionSilentlyAndWait(connection, server, region); 068 } 069 070 // Force ZK node to OFFLINE so master assigns 071 forceOfflineInZK(connection.getAdmin(), region); 072 } 073 074 /** 075 * Fix unassigned by creating/transition the unassigned ZK node for this region to OFFLINE state 076 * with a special flag to tell the master that this is a forced operation by HBCK. This assumes 077 * that info is in META. nnnn 078 */ 079 public static void fixUnassigned(Admin admin, RegionInfo region) 080 throws IOException, KeeperException, InterruptedException { 081 // Force ZK node to OFFLINE so master assigns 082 forceOfflineInZK(admin, region); 083 } 084 085 /** 086 * In 0.90, this forces an HRI offline by setting the RegionTransitionData in ZK to have 087 * HBCK_CODE_NAME as the server. This is a special case in the AssignmentManager that attempts an 088 * assign call by the master. This doesn't seem to work properly in the updated version of 0.92+'s 089 * hbck so we use assign to force the region into transition. This has the side-effect of 090 * requiring a RegionInfo that considers regionId (timestamp) in comparators that is addressed by 091 * HBASE-5563. 092 */ 093 private static void forceOfflineInZK(Admin admin, final RegionInfo region) 094 throws ZooKeeperConnectionException, KeeperException, IOException, InterruptedException { 095 admin.assign(region.getRegionName()); 096 } 097 098 /* 099 * Should we check all assignments or just not in RIT? 100 */ 101 public static void waitUntilAssigned(Admin admin, RegionInfo region) 102 throws IOException, InterruptedException { 103 long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000); 104 long expiration = timeout + EnvironmentEdgeManager.currentTime(); 105 while (EnvironmentEdgeManager.currentTime() < expiration) { 106 try { 107 boolean inTransition = false; 108 for (RegionState rs : admin.getClusterMetrics(EnumSet.of(Option.REGIONS_IN_TRANSITION)) 109 .getRegionStatesInTransition()) { 110 if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) { 111 inTransition = true; 112 break; 113 } 114 } 115 if (!inTransition) { 116 // yay! no longer RIT 117 return; 118 } 119 // still in rit 120 LOG.info("Region still in transition, waiting for " + "it to become assigned: " + region); 121 } catch (IOException e) { 122 LOG.warn("Exception when waiting for region to become assigned," + " retrying", e); 123 } 124 Thread.sleep(1000); 125 } 126 throw new IOException("Region " + region + " failed to move out of " 127 + "transition within timeout " + timeout + "ms"); 128 } 129 130 /** 131 * Contacts a region server and waits up to hbase.hbck.close.timeout ms (default 120s) to close 132 * the region. This bypasses the active hmaster. 133 */ 134 @SuppressWarnings("deprecation") 135 public static void closeRegionSilentlyAndWait(Connection connection, ServerName server, 136 RegionInfo region) throws IOException, InterruptedException { 137 long timeout = connection.getConfiguration().getLong("hbase.hbck.close.timeout", 120000); 138 ServerManager.closeRegionSilentlyAndWait((ClusterConnection) connection, server, region, 139 timeout); 140 } 141 142 /** 143 * Puts the specified RegionInfo into META with replica related columns 144 */ 145 public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf, RegionInfo hri, 146 Collection<ServerName> servers, int numReplicas) throws IOException { 147 Connection conn = ConnectionFactory.createConnection(conf); 148 Table meta = conn.getTable(TableName.META_TABLE_NAME); 149 Put put = MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime()); 150 if (numReplicas > 1) { 151 Random rand = ThreadLocalRandom.current(); 152 ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]); 153 for (int i = 1; i < numReplicas; i++) { 154 ServerName sn = serversArr[rand.nextInt(serversArr.length)]; 155 // the column added here is just to make sure the master is able to 156 // see the additional replicas when it is asked to assign. The 157 // final value of these columns will be different and will be updated 158 // by the actual regionservers that start hosting the respective replicas 159 MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), i); 160 } 161 } 162 meta.put(put); 163 meta.close(); 164 conn.close(); 165 } 166 167 /** 168 * Creates, flushes, and closes a new region. 169 */ 170 public static HRegion createHDFSRegionDir(Configuration conf, RegionInfo hri, TableDescriptor htd) 171 throws IOException { 172 // Create HRegion 173 Path root = CommonFSUtils.getRootDir(conf); 174 HRegion region = HRegion.createHRegion(hri, root, conf, htd, null); 175 176 // Close the new region to flush to disk. Close log file too. 177 region.close(); 178 return region; 179 } 180 181 /* 182 * Remove parent 183 */ 184 public static void removeParentInMeta(Configuration conf, RegionInfo hri) throws IOException { 185 Connection conn = ConnectionFactory.createConnection(conf); 186 MetaTableAccessor.deleteRegionInfo(conn, hri); 187 } 188}