HBASE-18503 Change ***Util and Master to use TableDescriptor and ColumnFamilyDescriptor
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / util / HBaseFsckRepair.java
blob6552b326206d5c5969a55f978ea0f00c87aed789
1 /**
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org.apache.hadoop.hbase.util;
21 import java.io.IOException;
22 import java.util.Collection;
23 import java.util.List;
24 import java.util.Random;
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.fs.Path;
30 import org.apache.hadoop.hbase.HRegionInfo;
31 import org.apache.hadoop.hbase.MetaTableAccessor;
32 import org.apache.hadoop.hbase.ServerName;
33 import org.apache.hadoop.hbase.TableName;
34 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.client.Admin;
37 import org.apache.hadoop.hbase.client.ClusterConnection;
38 import org.apache.hadoop.hbase.client.Connection;
39 import org.apache.hadoop.hbase.client.ConnectionFactory;
40 import org.apache.hadoop.hbase.client.Put;
41 import org.apache.hadoop.hbase.client.Table;
42 import org.apache.hadoop.hbase.client.TableDescriptor;
43 import org.apache.hadoop.hbase.master.RegionState;
44 import org.apache.hadoop.hbase.master.ServerManager;
45 import org.apache.hadoop.hbase.regionserver.HRegion;
46 import org.apache.zookeeper.KeeperException;
48 /**
49 * This class contains helper methods that repair parts of hbase's filesystem
50 * contents.
52 @InterfaceAudience.Private
53 public class HBaseFsckRepair {
54 private static final Log LOG = LogFactory.getLog(HBaseFsckRepair.class);
56 /**
57 * Fix multiple assignment by doing silent closes on each RS hosting the region
58 * and then force ZK unassigned node to OFFLINE to trigger assignment by
59 * master.
61 * @param connection HBase connection to the cluster
62 * @param region Region to undeploy
63 * @param servers list of Servers to undeploy from
65 public static void fixMultiAssignment(Connection connection, HRegionInfo region,
66 List<ServerName> servers)
67 throws IOException, KeeperException, InterruptedException {
68 HRegionInfo actualRegion = new HRegionInfo(region);
70 // Close region on the servers silently
71 for(ServerName server : servers) {
72 closeRegionSilentlyAndWait(connection, server, actualRegion);
75 // Force ZK node to OFFLINE so master assigns
76 forceOfflineInZK(connection.getAdmin(), actualRegion);
79 /**
80 * Fix unassigned by creating/transition the unassigned ZK node for this
81 * region to OFFLINE state with a special flag to tell the master that this is
82 * a forced operation by HBCK.
84 * This assumes that info is in META.
86 * @param admin
87 * @param region
88 * @throws IOException
89 * @throws KeeperException
91 public static void fixUnassigned(Admin admin, HRegionInfo region)
92 throws IOException, KeeperException, InterruptedException {
93 HRegionInfo actualRegion = new HRegionInfo(region);
95 // Force ZK node to OFFLINE so master assigns
96 forceOfflineInZK(admin, actualRegion);
99 /**
100 * In 0.90, this forces an HRI offline by setting the RegionTransitionData
101 * in ZK to have HBCK_CODE_NAME as the server. This is a special case in
102 * the AssignmentManager that attempts an assign call by the master.
104 * @see org.apache.hadoop.hbase.master.AssignementManager#handleHBCK
106 * This doesn't seem to work properly in the updated version of 0.92+'s hbck
107 * so we use assign to force the region into transition. This has the
108 * side-effect of requiring a HRegionInfo that considers regionId (timestamp)
109 * in comparators that is addressed by HBASE-5563.
111 private static void forceOfflineInZK(Admin admin, final HRegionInfo region)
112 throws ZooKeeperConnectionException, KeeperException, IOException, InterruptedException {
113 admin.assign(region.getRegionName());
117 * Should we check all assignments or just not in RIT?
119 public static void waitUntilAssigned(Admin admin,
120 HRegionInfo region) throws IOException, InterruptedException {
121 long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
122 long expiration = timeout + EnvironmentEdgeManager.currentTime();
123 while (EnvironmentEdgeManager.currentTime() < expiration) {
124 try {
125 boolean inTransition = false;
126 for (RegionState rs: admin.getClusterStatus().getRegionsInTransition()) {
127 if (rs.getRegion().equals(region)) {
128 inTransition = true;
129 break;
132 if (!inTransition) {
133 // yay! no longer RIT
134 return;
136 // still in rit
137 LOG.info("Region still in transition, waiting for "
138 + "it to become assigned: " + region);
139 } catch (IOException e) {
140 LOG.warn("Exception when waiting for region to become assigned,"
141 + " retrying", e);
143 Thread.sleep(1000);
145 throw new IOException("Region " + region + " failed to move out of " +
146 "transition within timeout " + timeout + "ms");
150 * Contacts a region server and waits up to hbase.hbck.close.timeout ms
151 * (default 120s) to close the region. This bypasses the active hmaster.
153 @SuppressWarnings("deprecation")
154 public static void closeRegionSilentlyAndWait(Connection connection,
155 ServerName server, HRegionInfo region) throws IOException, InterruptedException {
156 long timeout = connection.getConfiguration()
157 .getLong("hbase.hbck.close.timeout", 120000);
158 ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server,
159 region, timeout);
163 * Puts the specified HRegionInfo into META with replica related columns
165 public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf,
166 HRegionInfo hri, Collection<ServerName> servers, int numReplicas) throws IOException {
167 Connection conn = ConnectionFactory.createConnection(conf);
168 Table meta = conn.getTable(TableName.META_TABLE_NAME);
169 Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
170 if (numReplicas > 1) {
171 Random r = new Random();
172 ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]);
173 for (int i = 1; i < numReplicas; i++) {
174 ServerName sn = serversArr[r.nextInt(serversArr.length)];
175 // the column added here is just to make sure the master is able to
176 // see the additional replicas when it is asked to assign. The
177 // final value of these columns will be different and will be updated
178 // by the actual regionservers that start hosting the respective replicas
179 MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, i);
182 meta.put(put);
183 meta.close();
184 conn.close();
188 * Creates, flushes, and closes a new region.
190 public static HRegion createHDFSRegionDir(Configuration conf,
191 HRegionInfo hri, TableDescriptor htd) throws IOException {
192 // Create HRegion
193 Path root = FSUtils.getRootDir(conf);
194 HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
196 // Close the new region to flush to disk. Close log file too.
197 region.close();
198 return region;
202 * Remove parent
204 public static void removeParentInMeta(Configuration conf, HRegionInfo hri) throws IOException {
205 Connection conn = ConnectionFactory.createConnection(conf);
206 MetaTableAccessor.deleteRegion(conn, hri);