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}