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.Closeable;
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.io.PrintWriter;
025import java.io.StringWriter;
026import java.net.InetAddress;
027import java.net.URI;
028import java.util.ArrayList;
029import java.util.Collection;
030import java.util.Collections;
031import java.util.Comparator;
032import java.util.EnumSet;
033import java.util.HashMap;
034import java.util.HashSet;
035import java.util.Iterator;
036import java.util.List;
037import java.util.Locale;
038import java.util.Map;
039import java.util.Map.Entry;
040import java.util.Objects;
041import java.util.Optional;
042import java.util.Set;
043import java.util.SortedMap;
044import java.util.TreeMap;
045import java.util.Vector;
046import java.util.concurrent.Callable;
047import java.util.concurrent.ConcurrentSkipListMap;
048import java.util.concurrent.ExecutionException;
049import java.util.concurrent.ExecutorService;
050import java.util.concurrent.Executors;
051import java.util.concurrent.Future;
052import java.util.concurrent.FutureTask;
053import java.util.concurrent.ScheduledThreadPoolExecutor;
054import java.util.concurrent.TimeUnit;
055import java.util.concurrent.TimeoutException;
056import java.util.concurrent.atomic.AtomicBoolean;
057import java.util.concurrent.atomic.AtomicInteger;
058import java.util.stream.Collectors;
059import org.apache.commons.io.IOUtils;
060import org.apache.commons.lang3.StringUtils;
061import org.apache.hadoop.conf.Configuration;
062import org.apache.hadoop.conf.Configured;
063import org.apache.hadoop.fs.FSDataOutputStream;
064import org.apache.hadoop.fs.FileStatus;
065import org.apache.hadoop.fs.FileSystem;
066import org.apache.hadoop.fs.Path;
067import org.apache.hadoop.fs.permission.FsAction;
068import org.apache.hadoop.fs.permission.FsPermission;
069import org.apache.hadoop.hbase.Abortable;
070import org.apache.hadoop.hbase.Cell;
071import org.apache.hadoop.hbase.CellUtil;
072import org.apache.hadoop.hbase.ClusterMetrics;
073import org.apache.hadoop.hbase.ClusterMetrics.Option;
074import org.apache.hadoop.hbase.HBaseConfiguration;
075import org.apache.hadoop.hbase.HBaseInterfaceAudience;
076import org.apache.hadoop.hbase.HConstants;
077import org.apache.hadoop.hbase.HRegionLocation;
078import org.apache.hadoop.hbase.KeyValue;
079import org.apache.hadoop.hbase.MasterNotRunningException;
080import org.apache.hadoop.hbase.MetaTableAccessor;
081import org.apache.hadoop.hbase.RegionLocations;
082import org.apache.hadoop.hbase.ServerName;
083import org.apache.hadoop.hbase.TableName;
084import org.apache.hadoop.hbase.TableNotFoundException;
085import org.apache.hadoop.hbase.ZooKeeperConnectionException;
086import org.apache.hadoop.hbase.client.Admin;
087import org.apache.hadoop.hbase.client.ClusterConnection;
088import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
089import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
090import org.apache.hadoop.hbase.client.Connection;
091import org.apache.hadoop.hbase.client.ConnectionFactory;
092import org.apache.hadoop.hbase.client.Delete;
093import org.apache.hadoop.hbase.client.Get;
094import org.apache.hadoop.hbase.client.Put;
095import org.apache.hadoop.hbase.client.RegionInfo;
096import org.apache.hadoop.hbase.client.RegionInfoBuilder;
097import org.apache.hadoop.hbase.client.RegionReplicaUtil;
098import org.apache.hadoop.hbase.client.Result;
099import org.apache.hadoop.hbase.client.ResultScanner;
100import org.apache.hadoop.hbase.client.RowMutations;
101import org.apache.hadoop.hbase.client.Scan;
102import org.apache.hadoop.hbase.client.Table;
103import org.apache.hadoop.hbase.client.TableDescriptor;
104import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
105import org.apache.hadoop.hbase.client.TableState;
106import org.apache.hadoop.hbase.io.FileLink;
107import org.apache.hadoop.hbase.io.HFileLink;
108import org.apache.hadoop.hbase.io.hfile.CacheConfig;
109import org.apache.hadoop.hbase.io.hfile.HFile;
110import org.apache.hadoop.hbase.master.RegionState;
111import org.apache.hadoop.hbase.regionserver.HRegion;
112import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
113import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
114import org.apache.hadoop.hbase.replication.ReplicationException;
115import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
116import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
117import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
118import org.apache.hadoop.hbase.security.UserProvider;
119import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
120import org.apache.hadoop.hbase.util.HbckErrorReporter.ERROR_CODE;
121import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
122import org.apache.hadoop.hbase.util.hbck.ReplicationChecker;
123import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
124import org.apache.hadoop.hbase.wal.WALSplitUtil;
125import org.apache.hadoop.hbase.zookeeper.ZKUtil;
126import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
127import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
128import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
129import org.apache.hadoop.ipc.RemoteException;
130import org.apache.hadoop.security.AccessControlException;
131import org.apache.hadoop.security.UserGroupInformation;
132import org.apache.hadoop.util.ReflectionUtils;
133import org.apache.hadoop.util.Tool;
134import org.apache.hadoop.util.ToolRunner;
135import org.apache.yetus.audience.InterfaceAudience;
136import org.apache.yetus.audience.InterfaceStability;
137import org.apache.zookeeper.KeeperException;
138import org.slf4j.Logger;
139import org.slf4j.LoggerFactory;
140
141import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
142import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
143import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
144import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
145import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
146import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
147
148import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
149import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
150
151/**
152 * HBaseFsck (hbck) is a tool for checking and repairing region consistency and table integrity
153 * problems in a corrupted HBase. This tool was written for hbase-1.x. It does not work with
154 * hbase-2.x; it can read state but is not allowed to change state; i.e. effect 'repair'. Even
155 * though it can 'read' state, given how so much has changed in how hbase1 and hbase2 operate, it
156 * will often misread. See hbck2 (HBASE-19121) for a hbck tool for hbase2. This class is deprecated.
157 * <p>
158 * Region consistency checks verify that hbase:meta, region deployment on region servers and the
159 * state of data in HDFS (.regioninfo files) all are in accordance.
160 * <p>
161 * Table integrity checks verify that all possible row keys resolve to exactly one region of a
162 * table. This means there are no individual degenerate or backwards regions; no holes between
163 * regions; and that there are no overlapping regions.
164 * <p>
165 * The general repair strategy works in two phases:
166 * <ol>
167 * <li>Repair Table Integrity on HDFS. (merge or fabricate regions)
168 * <li>Repair Region Consistency with hbase:meta and assignments
169 * </ol>
170 * <p>
171 * For table integrity repairs, the tables' region directories are scanned for .regioninfo files.
172 * Each table's integrity is then verified. If there are any orphan regions (regions with no
173 * .regioninfo files) or holes, new regions are fabricated. Backwards regions are sidelined as well
174 * as empty degenerate (endkey==startkey) regions. If there are any overlapping regions, a new
175 * region is created and all data is merged into the new region.
176 * <p>
177 * Table integrity repairs deal solely with HDFS and could potentially be done offline -- the hbase
178 * region servers or master do not need to be running. This phase can eventually be used to
179 * completely reconstruct the hbase:meta table in an offline fashion.
180 * <p>
181 * Region consistency requires three conditions -- 1) valid .regioninfo file present in an HDFS
182 * region dir, 2) valid row with .regioninfo data in META, and 3) a region is deployed only at the
183 * regionserver that was assigned to with proper state in the master.
184 * <p>
185 * Region consistency repairs require hbase to be online so that hbck can contact the HBase master
186 * and region servers. The hbck#connect() method must first be called successfully. Much of the
187 * region consistency information is transient and less risky to repair.
188 * <p>
189 * If hbck is run from the command line, there are a handful of arguments that can be used to limit
190 * the kinds of repairs hbck will do. See the code in {@link #printUsageAndExit()} for more details.
191 * @deprecated For removal in hbase-4.0.0. Use HBCK2 instead.
192 */
193@Deprecated
194@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
195@InterfaceStability.Evolving
196public class HBaseFsck extends Configured implements Closeable {
197  public static final long DEFAULT_TIME_LAG = 60000; // default value of 1 minute
198  public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000;
199  private static final int MAX_NUM_THREADS = 50; // #threads to contact regions
200  private static boolean rsSupportsOffline = true;
201  private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2;
202  private static final int DEFAULT_MAX_MERGE = 5;
203
204  /**
205   * Here is where hbase-1.x used to default the lock for hbck1. It puts in place a lock when it
206   * goes to write/make changes.
207   */
208  @InterfaceAudience.Private
209  public static final String HBCK_LOCK_FILE = "hbase-hbck.lock";
210
211  private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5;
212  private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds
213  private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds
214  // We have to set the timeout value > HdfsConstants.LEASE_SOFTLIMIT_PERIOD.
215  // In HADOOP-2.6 and later, the Namenode proxy now created with custom RetryPolicy for
216  // AlreadyBeingCreatedException which is implies timeout on this operations up to
217  // HdfsConstants.LEASE_SOFTLIMIT_PERIOD (60 seconds).
218  private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT = 80; // seconds
219  private static final int DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS = 5;
220  private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds
221  private static final int DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds
222
223  /**********************
224   * Internal resources
225   **********************/
226  private static final Logger LOG = LoggerFactory.getLogger(HBaseFsck.class.getName());
227  private ClusterMetrics status;
228  private ClusterConnection connection;
229  private Admin admin;
230  private Table meta;
231  // threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions
232  protected ExecutorService executor;
233  private long startMillis = EnvironmentEdgeManager.currentTime();
234  private HFileCorruptionChecker hfcc;
235  private int retcode = 0;
236  private Path HBCK_LOCK_PATH;
237  private FSDataOutputStream hbckOutFd;
238  // This lock is to prevent cleanup of balancer resources twice between
239  // ShutdownHook and the main code. We cleanup only if the connect() is
240  // successful
241  private final AtomicBoolean hbckLockCleanup = new AtomicBoolean(false);
242
243  // Unsupported options in HBase 2.0+
244  private static final Set<String> unsupportedOptionsInV2 = Sets.newHashSet("-fix",
245    "-fixAssignments", "-fixMeta", "-fixHdfsHoles", "-fixHdfsOrphans", "-fixTableOrphans",
246    "-fixHdfsOverlaps", "-sidelineBigOverlaps", "-fixSplitParents", "-removeParents",
247    "-fixEmptyMetaCells", "-repair", "-repairHoles", "-maxOverlapsToSideline", "-maxMerge");
248
249  /***********
250   * Options
251   ***********/
252  private static boolean details = false; // do we display the full report
253  private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
254  private static boolean forceExclusive = false; // only this hbck can modify HBase
255  private boolean fixAssignments = false; // fix assignment errors?
256  private boolean fixMeta = false; // fix meta errors?
257  private boolean checkHdfs = true; // load and check fs consistency?
258  private boolean fixHdfsHoles = false; // fix fs holes?
259  private boolean fixHdfsOverlaps = false; // fix fs overlaps (risky)
260  private boolean fixHdfsOrphans = false; // fix fs holes (missing .regioninfo)
261  private boolean fixTableOrphans = false; // fix fs holes (missing .tableinfo)
262  private boolean fixVersionFile = false; // fix missing hbase.version file in hdfs
263  private boolean fixSplitParents = false; // fix lingering split parents
264  private boolean removeParents = false; // remove split parents
265  private boolean fixReferenceFiles = false; // fix lingering reference store file
266  private boolean fixHFileLinks = false; // fix lingering HFileLinks
267  private boolean fixEmptyMetaCells = false; // fix (remove) empty REGIONINFO_QUALIFIER rows
268  private boolean fixReplication = false; // fix undeleted replication queues for removed peer
269  private boolean cleanReplicationBarrier = false; // clean replication barriers of a table
270  private boolean fixAny = false; // Set to true if any of the fix is required.
271
272  // limit checking/fixes to listed tables, if empty attempt to check/fix all
273  // hbase:meta are always checked
274  private Set<TableName> tablesIncluded = new HashSet<>();
275  private TableName cleanReplicationBarrierTable;
276  private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge
277  // maximum number of overlapping regions to sideline
278  private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE;
279  private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions
280  private Path sidelineDir = null;
281
282  private boolean rerun = false; // if we tried to fix something, rerun hbck
283  private static boolean summary = false; // if we want to print less output
284  private boolean checkMetaOnly = false;
285  private boolean checkRegionBoundaries = false;
286  private boolean ignorePreCheckPermission = false; // if pre-check permission
287
288  /*********
289   * State
290   *********/
291  final private HbckErrorReporter errors;
292  int fixes = 0;
293
294  /**
295   * This map contains the state of all hbck items. It maps from encoded region name to
296   * HbckRegionInfo structure. The information contained in HbckRegionInfo is used to detect and
297   * correct consistency (hdfs/meta/deployment) problems.
298   */
299  private TreeMap<String, HbckRegionInfo> regionInfoMap = new TreeMap<>();
300  // Empty regioninfo qualifiers in hbase:meta
301  private Set<Result> emptyRegionInfoQualifiers = new HashSet<>();
302
303  /**
304   * This map from Tablename -> TableInfo contains the structures necessary to detect table
305   * consistency problems (holes, dupes, overlaps). It is sorted to prevent dupes. If tablesIncluded
306   * is empty, this map contains all tables. Otherwise, it contains only meta tables and tables in
307   * tablesIncluded, unless checkMetaOnly is specified, in which case, it contains only the meta
308   * table
309   */
310  private SortedMap<TableName, HbckTableInfo> tablesInfo = new ConcurrentSkipListMap<>();
311
312  /**
313   * When initially looking at HDFS, we attempt to find any orphaned data.
314   */
315  private List<HbckRegionInfo> orphanHdfsDirs = Collections.synchronizedList(new ArrayList<>());
316
317  private Map<TableName, Set<String>> orphanTableDirs = new HashMap<>();
318  private Map<TableName, TableState> tableStates = new HashMap<>();
319  private final RetryCounterFactory lockFileRetryCounterFactory;
320  private final RetryCounterFactory createZNodeRetryCounterFactory;
321
322  private Map<TableName, Set<String>> skippedRegions = new HashMap<>();
323
324  private ZKWatcher zkw = null;
325  private String hbckEphemeralNodePath = null;
326  private boolean hbckZodeCreated = false;
327
328  /**
329   * Constructor
330   * @param conf Configuration object
331   * @throws MasterNotRunningException    if the master is not running
332   * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper
333   */
334  public HBaseFsck(Configuration conf) throws IOException, ClassNotFoundException {
335    this(conf, createThreadPool(conf));
336  }
337
338  private static ExecutorService createThreadPool(Configuration conf) {
339    int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
340    return new ScheduledThreadPoolExecutor(numThreads,
341      new ThreadFactoryBuilder().setNameFormat("hbasefsck-pool-%d").setDaemon(true)
342        .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
343  }
344
345  /**
346   * Constructor Configuration object if the master is not running if unable to connect to ZooKeeper
347   */
348  public HBaseFsck(Configuration conf, ExecutorService exec) throws MasterNotRunningException,
349    ZooKeeperConnectionException, IOException, ClassNotFoundException {
350    super(conf);
351    errors = getErrorReporter(getConf());
352    this.executor = exec;
353    lockFileRetryCounterFactory = createLockRetryCounterFactory(getConf());
354    createZNodeRetryCounterFactory = createZnodeRetryCounterFactory(getConf());
355    zkw = createZooKeeperWatcher();
356  }
357
358  /** Returns A retry counter factory configured for retrying lock file creation. */
359  public static RetryCounterFactory createLockRetryCounterFactory(Configuration conf) {
360    return new RetryCounterFactory(
361      conf.getInt("hbase.hbck.lockfile.attempts", DEFAULT_MAX_LOCK_FILE_ATTEMPTS),
362      conf.getInt("hbase.hbck.lockfile.attempt.sleep.interval",
363        DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL),
364      conf.getInt("hbase.hbck.lockfile.attempt.maxsleeptime",
365        DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME));
366  }
367
368  /** Returns A retry counter factory configured for retrying znode creation. */
369  private static RetryCounterFactory createZnodeRetryCounterFactory(Configuration conf) {
370    return new RetryCounterFactory(
371      conf.getInt("hbase.hbck.createznode.attempts", DEFAULT_MAX_CREATE_ZNODE_ATTEMPTS),
372      conf.getInt("hbase.hbck.createznode.attempt.sleep.interval",
373        DEFAULT_CREATE_ZNODE_ATTEMPT_SLEEP_INTERVAL),
374      conf.getInt("hbase.hbck.createznode.attempt.maxsleeptime",
375        DEFAULT_CREATE_ZNODE_ATTEMPT_MAX_SLEEP_TIME));
376  }
377
378  /** Returns Return the tmp dir this tool writes too. */
379  @InterfaceAudience.Private
380  public static Path getTmpDir(Configuration conf) throws IOException {
381    return new Path(CommonFSUtils.getRootDir(conf), HConstants.HBASE_TEMP_DIRECTORY);
382  }
383
384  private static class FileLockCallable implements Callable<FSDataOutputStream> {
385    RetryCounter retryCounter;
386    private final Configuration conf;
387    private Path hbckLockPath = null;
388
389    public FileLockCallable(Configuration conf, RetryCounter retryCounter) {
390      this.retryCounter = retryCounter;
391      this.conf = conf;
392    }
393
394    /** Returns Will be <code>null</code> unless you call {@link #call()} */
395    Path getHbckLockPath() {
396      return this.hbckLockPath;
397    }
398
399    @Override
400    public FSDataOutputStream call() throws IOException {
401      try {
402        FileSystem fs = CommonFSUtils.getCurrentFileSystem(this.conf);
403        FsPermission defaultPerms =
404          CommonFSUtils.getFilePermissions(fs, this.conf, HConstants.DATA_FILE_UMASK_KEY);
405        Path tmpDir = getTmpDir(conf);
406        this.hbckLockPath = new Path(tmpDir, HBCK_LOCK_FILE);
407        fs.mkdirs(tmpDir);
408        final FSDataOutputStream out = createFileWithRetries(fs, this.hbckLockPath, defaultPerms);
409        out.writeBytes(InetAddress.getLocalHost().toString());
410        // Add a note into the file we write on why hbase2 is writing out an hbck1 lock file.
411        out.writeBytes(" Written by an hbase-2.x Master to block an "
412          + "attempt by an hbase-1.x HBCK tool making modification to state. "
413          + "See 'HBCK must match HBase server version' in the hbase refguide.");
414        out.flush();
415        return out;
416      } catch (RemoteException e) {
417        if (AlreadyBeingCreatedException.class.getName().equals(e.getClassName())) {
418          return null;
419        } else {
420          throw e;
421        }
422      }
423    }
424
425    private FSDataOutputStream createFileWithRetries(final FileSystem fs,
426      final Path hbckLockFilePath, final FsPermission defaultPerms) throws IOException {
427      IOException exception = null;
428      do {
429        try {
430          return CommonFSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
431        } catch (IOException ioe) {
432          LOG.info("Failed to create lock file " + hbckLockFilePath.getName() + ", try="
433            + (retryCounter.getAttemptTimes() + 1) + " of " + retryCounter.getMaxAttempts());
434          LOG.debug("Failed to create lock file " + hbckLockFilePath.getName(), ioe);
435          try {
436            exception = ioe;
437            retryCounter.sleepUntilNextRetry();
438          } catch (InterruptedException ie) {
439            throw (InterruptedIOException) new InterruptedIOException(
440              "Can't create lock file " + hbckLockFilePath.getName()).initCause(ie);
441          }
442        }
443      } while (retryCounter.shouldRetry());
444
445      throw exception;
446    }
447  }
448
449  /**
450   * This method maintains a lock using a file. If the creation fails we return null
451   * @return FSDataOutputStream object corresponding to the newly opened lock file
452   * @throws IOException if IO failure occurs
453   */
454  public static Pair<Path, FSDataOutputStream> checkAndMarkRunningHbck(Configuration conf,
455    RetryCounter retryCounter) throws IOException {
456    FileLockCallable callable = new FileLockCallable(conf, retryCounter);
457    ExecutorService executor = Executors.newFixedThreadPool(1);
458    FutureTask<FSDataOutputStream> futureTask = new FutureTask<>(callable);
459    executor.execute(futureTask);
460    final int timeoutInSeconds =
461      conf.getInt("hbase.hbck.lockfile.maxwaittime", DEFAULT_WAIT_FOR_LOCK_TIMEOUT);
462    FSDataOutputStream stream = null;
463    try {
464      stream = futureTask.get(timeoutInSeconds, TimeUnit.SECONDS);
465    } catch (ExecutionException ee) {
466      LOG.warn("Encountered exception when opening lock file", ee);
467    } catch (InterruptedException ie) {
468      LOG.warn("Interrupted when opening lock file", ie);
469      Thread.currentThread().interrupt();
470    } catch (TimeoutException exception) {
471      // took too long to obtain lock
472      LOG.warn("Took more than " + timeoutInSeconds + " seconds in obtaining lock");
473      futureTask.cancel(true);
474    } finally {
475      executor.shutdownNow();
476    }
477    return new Pair<Path, FSDataOutputStream>(callable.getHbckLockPath(), stream);
478  }
479
480  private void unlockHbck() {
481    if (isExclusive() && hbckLockCleanup.compareAndSet(true, false)) {
482      RetryCounter retryCounter = lockFileRetryCounterFactory.create();
483      do {
484        try {
485          Closeables.close(hbckOutFd, true);
486          CommonFSUtils.delete(CommonFSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
487          LOG.info("Finishing hbck");
488          return;
489        } catch (IOException ioe) {
490          LOG.info("Failed to delete " + HBCK_LOCK_PATH + ", try="
491            + (retryCounter.getAttemptTimes() + 1) + " of " + retryCounter.getMaxAttempts());
492          LOG.debug("Failed to delete " + HBCK_LOCK_PATH, ioe);
493          try {
494            retryCounter.sleepUntilNextRetry();
495          } catch (InterruptedException ie) {
496            Thread.currentThread().interrupt();
497            LOG.warn("Interrupted while deleting lock file" + HBCK_LOCK_PATH);
498            return;
499          }
500        }
501      } while (retryCounter.shouldRetry());
502    }
503  }
504
505  /**
506   * To repair region consistency, one must call connect() in order to repair online state.
507   */
508  public void connect() throws IOException {
509
510    if (isExclusive()) {
511      // Grab the lock
512      Pair<Path, FSDataOutputStream> pair =
513        checkAndMarkRunningHbck(getConf(), this.lockFileRetryCounterFactory.create());
514      HBCK_LOCK_PATH = pair.getFirst();
515      this.hbckOutFd = pair.getSecond();
516      if (hbckOutFd == null) {
517        setRetCode(-1);
518        LOG.error("Another instance of hbck is fixing HBase, exiting this instance. "
519          + "[If you are sure no other instance is running, delete the lock file " + HBCK_LOCK_PATH
520          + " and rerun the tool]");
521        throw new IOException("Duplicate hbck - Abort");
522      }
523
524      // Make sure to cleanup the lock
525      hbckLockCleanup.set(true);
526    }
527
528    // Add a shutdown hook to this thread, in case user tries to
529    // kill the hbck with a ctrl-c, we want to cleanup the lock so that
530    // it is available for further calls
531    Runtime.getRuntime().addShutdownHook(new Thread() {
532      @Override
533      public void run() {
534        IOUtils.closeQuietly(HBaseFsck.this, e -> LOG.warn("", e));
535        cleanupHbckZnode();
536        unlockHbck();
537      }
538    });
539
540    LOG.info("Launching hbck");
541
542    connection = (ClusterConnection) ConnectionFactory.createConnection(getConf());
543    admin = connection.getAdmin();
544    meta = connection.getTable(TableName.META_TABLE_NAME);
545    status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS,
546      Option.MASTER, Option.BACKUP_MASTERS, Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION));
547  }
548
549  /**
550   * Get deployed regions according to the region servers.
551   */
552  private void loadDeployedRegions() throws IOException, InterruptedException {
553    // From the master, get a list of all known live region servers
554    Collection<ServerName> regionServers = status.getLiveServerMetrics().keySet();
555    errors.print("Number of live region servers: " + regionServers.size());
556    if (details) {
557      for (ServerName rsinfo : regionServers) {
558        errors.print("  " + rsinfo.getServerName());
559      }
560    }
561
562    // From the master, get a list of all dead region servers
563    Collection<ServerName> deadRegionServers = status.getDeadServerNames();
564    errors.print("Number of dead region servers: " + deadRegionServers.size());
565    if (details) {
566      for (ServerName name : deadRegionServers) {
567        errors.print("  " + name);
568      }
569    }
570
571    // Print the current master name and state
572    errors.print("Master: " + status.getMasterName());
573
574    // Print the list of all backup masters
575    Collection<ServerName> backupMasters = status.getBackupMasterNames();
576    errors.print("Number of backup masters: " + backupMasters.size());
577    if (details) {
578      for (ServerName name : backupMasters) {
579        errors.print("  " + name);
580      }
581    }
582
583    errors.print("Average load: " + status.getAverageLoad());
584    errors.print("Number of requests: " + status.getRequestCount());
585    errors.print("Number of regions: " + status.getRegionCount());
586
587    List<RegionState> rits = status.getRegionStatesInTransition();
588    errors.print("Number of regions in transition: " + rits.size());
589    if (details) {
590      for (RegionState state : rits) {
591        errors.print("  " + state.toDescriptiveString());
592      }
593    }
594
595    // Determine what's deployed
596    processRegionServers(regionServers);
597  }
598
599  /**
600   * Clear the current state of hbck.
601   */
602  private void clearState() {
603    // Make sure regionInfo is empty before starting
604    fixes = 0;
605    regionInfoMap.clear();
606    emptyRegionInfoQualifiers.clear();
607    tableStates.clear();
608    errors.clear();
609    tablesInfo.clear();
610    orphanHdfsDirs.clear();
611    skippedRegions.clear();
612  }
613
614  /**
615   * This repair method analyzes hbase data in hdfs and repairs it to satisfy the table integrity
616   * rules. HBase doesn't need to be online for this operation to work.
617   */
618  public void offlineHdfsIntegrityRepair() throws IOException, InterruptedException {
619    // Initial pass to fix orphans.
620    if (
621      shouldCheckHdfs() && (shouldFixHdfsOrphans() || shouldFixHdfsHoles()
622        || shouldFixHdfsOverlaps() || shouldFixTableOrphans())
623    ) {
624      LOG.info("Loading regioninfos HDFS");
625      // if nothing is happening this should always complete in two iterations.
626      int maxIterations = getConf().getInt("hbase.hbck.integrityrepair.iterations.max", 3);
627      int curIter = 0;
628      do {
629        clearState(); // clears hbck state and reset fixes to 0 and.
630        // repair what's on HDFS
631        restoreHdfsIntegrity();
632        curIter++;// limit the number of iterations.
633      } while (fixes > 0 && curIter <= maxIterations);
634
635      // Repairs should be done in the first iteration and verification in the second.
636      // If there are more than 2 passes, something funny has happened.
637      if (curIter > 2) {
638        if (curIter == maxIterations) {
639          LOG.warn("Exiting integrity repairs after max " + curIter + " iterations. "
640            + "Tables integrity may not be fully repaired!");
641        } else {
642          LOG.info("Successfully exiting integrity repairs after " + curIter + " iterations");
643        }
644      }
645    }
646  }
647
648  /**
649   * This repair method requires the cluster to be online since it contacts region servers and the
650   * masters. It makes each region's state in HDFS, in hbase:meta, and deployments consistent.
651   * @return If &gt; 0 , number of errors detected, if &lt; 0 there was an unrecoverable error. If
652   *         0, we have a clean hbase.
653   */
654  public int onlineConsistencyRepair() throws IOException, KeeperException, InterruptedException {
655
656    // get regions according to what is online on each RegionServer
657    loadDeployedRegions();
658    // check whether hbase:meta is deployed and online
659    recordMetaRegion();
660    // Report inconsistencies if there are any unknown servers.
661    reportUnknownServers();
662    // Check if hbase:meta is found only once and in the right place
663    if (!checkMetaRegion()) {
664      String errorMsg = "hbase:meta table is not consistent. ";
665      if (shouldFixAssignments()) {
666        errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state.";
667      } else {
668        errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency.";
669      }
670      errors.reportError(errorMsg + " Exiting...");
671      return -2;
672    }
673    // Not going with further consistency check for tables when hbase:meta itself is not consistent.
674    LOG.info("Loading regionsinfo from the hbase:meta table");
675    boolean success = loadMetaEntries();
676    if (!success) return -1;
677
678    // Empty cells in hbase:meta?
679    reportEmptyMetaCells();
680
681    // Check if we have to cleanup empty REGIONINFO_QUALIFIER rows from hbase:meta
682    if (shouldFixEmptyMetaCells()) {
683      fixEmptyMetaCells();
684    }
685
686    // get a list of all tables that have not changed recently.
687    if (!checkMetaOnly) {
688      reportTablesInFlux();
689    }
690
691    // Get disabled tables states
692    loadTableStates();
693
694    // load regiondirs and regioninfos from HDFS
695    if (shouldCheckHdfs()) {
696      LOG.info("Loading region directories from HDFS");
697      loadHdfsRegionDirs();
698      LOG.info("Loading region information from HDFS");
699      loadHdfsRegionInfos();
700    }
701
702    // fix the orphan tables
703    fixOrphanTables();
704
705    LOG.info("Checking and fixing region consistency");
706    // Check and fix consistency
707    checkAndFixConsistency();
708
709    // Check integrity (does not fix)
710    checkIntegrity();
711    return errors.getErrorList().size();
712  }
713
714  private void reportUnknownServers() throws IOException {
715    List<ServerName> unknownServers = admin.listUnknownServers();
716    if (!unknownServers.isEmpty()) {
717      unknownServers.stream().forEach(serverName -> {
718        errors.reportError(ERROR_CODE.UNKNOWN_SERVER,
719          "Found unknown server, some of the regions held by this server may not get assigned. "
720            + String.format("Use HBCK2 scheduleRecoveries %s to recover.", serverName));
721      });
722    }
723  }
724
725  /**
726   * This method maintains an ephemeral znode. If the creation fails we return false or throw
727   * exception
728   * @return true if creating znode succeeds; false otherwise
729   * @throws IOException if IO failure occurs
730   */
731  private boolean setMasterInMaintenanceMode() throws IOException {
732    RetryCounter retryCounter = createZNodeRetryCounterFactory.create();
733    hbckEphemeralNodePath = ZNodePaths.joinZNode(zkw.getZNodePaths().masterMaintZNode,
734      "hbck-" + Long.toString(EnvironmentEdgeManager.currentTime()));
735    do {
736      try {
737        hbckZodeCreated = ZKUtil.createEphemeralNodeAndWatch(zkw, hbckEphemeralNodePath, null);
738        if (hbckZodeCreated) {
739          break;
740        }
741      } catch (KeeperException e) {
742        if (retryCounter.getAttemptTimes() >= retryCounter.getMaxAttempts()) {
743          throw new IOException("Can't create znode " + hbckEphemeralNodePath, e);
744        }
745        // fall through and retry
746      }
747
748      LOG.warn("Fail to create znode " + hbckEphemeralNodePath + ", try="
749        + (retryCounter.getAttemptTimes() + 1) + " of " + retryCounter.getMaxAttempts());
750
751      try {
752        retryCounter.sleepUntilNextRetry();
753      } catch (InterruptedException ie) {
754        throw (InterruptedIOException) new InterruptedIOException(
755          "Can't create znode " + hbckEphemeralNodePath).initCause(ie);
756      }
757    } while (retryCounter.shouldRetry());
758    return hbckZodeCreated;
759  }
760
761  private void cleanupHbckZnode() {
762    try {
763      if (zkw != null && hbckZodeCreated) {
764        ZKUtil.deleteNode(zkw, hbckEphemeralNodePath);
765        hbckZodeCreated = false;
766      }
767    } catch (KeeperException e) {
768      // Ignore
769      if (!e.code().equals(KeeperException.Code.NONODE)) {
770        LOG.warn("Delete HBCK znode " + hbckEphemeralNodePath + " failed ", e);
771      }
772    }
773  }
774
775  /**
776   * Contacts the master and prints out cluster-wide information
777   * @return 0 on success, non-zero on failure
778   */
779  public int onlineHbck()
780    throws IOException, KeeperException, InterruptedException, ReplicationException {
781    // print hbase server version
782    errors.print("Version: " + status.getHBaseVersion());
783
784    // Clean start
785    clearState();
786    // Do offline check and repair first
787    offlineHdfsIntegrityRepair();
788    offlineReferenceFileRepair();
789    offlineHLinkFileRepair();
790    // If Master runs maintenance tasks (such as balancer, catalog janitor, etc) during online
791    // hbck, it is likely that hbck would be misled and report transient errors. Therefore, it
792    // is better to set Master into maintenance mode during online hbck.
793    //
794    if (!setMasterInMaintenanceMode()) {
795      LOG.warn("HBCK is running while master is not in maintenance mode, you might see transient "
796        + "error.  Please run HBCK multiple times to reduce the chance of transient error.");
797    }
798
799    onlineConsistencyRepair();
800
801    if (checkRegionBoundaries) {
802      checkRegionBoundaries();
803    }
804
805    checkAndFixReplication();
806
807    cleanReplicationBarrier();
808
809    // Remove the hbck znode
810    cleanupHbckZnode();
811
812    // Remove the hbck lock
813    unlockHbck();
814
815    // Print table summary
816    printTableSummary(tablesInfo);
817    return errors.summarize();
818  }
819
820  public static byte[] keyOnly(byte[] b) {
821    if (b == null) return b;
822    int rowlength = Bytes.toShort(b, 0);
823    byte[] result = new byte[rowlength];
824    System.arraycopy(b, Bytes.SIZEOF_SHORT, result, 0, rowlength);
825    return result;
826  }
827
828  @Override
829  public void close() throws IOException {
830    try {
831      cleanupHbckZnode();
832      unlockHbck();
833    } catch (Exception io) {
834      LOG.warn(io.toString(), io);
835    } finally {
836      if (zkw != null) {
837        zkw.close();
838        zkw = null;
839      }
840      IOUtils.closeQuietly(admin, e -> LOG.warn("", e));
841      IOUtils.closeQuietly(meta, e -> LOG.warn("", e));
842      IOUtils.closeQuietly(connection, e -> LOG.warn("", e));
843    }
844  }
845
846  private static class RegionBoundariesInformation {
847    public byte[] regionName;
848    public byte[] metaFirstKey;
849    public byte[] metaLastKey;
850    public byte[] storesFirstKey;
851    public byte[] storesLastKey;
852
853    @Override
854    public String toString() {
855      return "regionName=" + Bytes.toStringBinary(regionName) + "\nmetaFirstKey="
856        + Bytes.toStringBinary(metaFirstKey) + "\nmetaLastKey=" + Bytes.toStringBinary(metaLastKey)
857        + "\nstoresFirstKey=" + Bytes.toStringBinary(storesFirstKey) + "\nstoresLastKey="
858        + Bytes.toStringBinary(storesLastKey);
859    }
860  }
861
862  public void checkRegionBoundaries() {
863    try {
864      ByteArrayComparator comparator = new ByteArrayComparator();
865      List<RegionInfo> regions = MetaTableAccessor.getAllRegions(connection, true);
866      final RegionBoundariesInformation currentRegionBoundariesInformation =
867        new RegionBoundariesInformation();
868      Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
869      for (RegionInfo regionInfo : regions) {
870        Path tableDir = CommonFSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
871        currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
872        // For each region, get the start and stop key from the META and compare them to the
873        // same information from the Stores.
874        Path path = new Path(tableDir, regionInfo.getEncodedName());
875        FileSystem fs = path.getFileSystem(getConf());
876        FileStatus[] files = fs.listStatus(path);
877        // For all the column families in this region...
878        byte[] storeFirstKey = null;
879        byte[] storeLastKey = null;
880        for (FileStatus file : files) {
881          String fileName = file.getPath().toString();
882          fileName = fileName.substring(fileName.lastIndexOf("/") + 1);
883          if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) {
884            FileStatus[] storeFiles = fs.listStatus(file.getPath());
885            // For all the stores in this column family.
886            for (FileStatus storeFile : storeFiles) {
887              HFile.Reader reader =
888                HFile.createReader(fs, storeFile.getPath(), CacheConfig.DISABLED, true, getConf());
889              if (
890                (reader.getFirstKey() != null)
891                  && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
892                    ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))
893              ) {
894                storeFirstKey = ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey();
895              }
896              if (
897                (reader.getLastKey() != null)
898                  && ((storeLastKey == null) || (comparator.compare(storeLastKey,
899                    ((KeyValue.KeyOnlyKeyValue) reader.getLastKey().get()).getKey())) < 0)
900              ) {
901                storeLastKey = ((KeyValue.KeyOnlyKeyValue) reader.getLastKey().get()).getKey();
902              }
903              reader.close();
904            }
905          }
906        }
907        currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey();
908        currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey();
909        currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey);
910        currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey);
911        if (currentRegionBoundariesInformation.metaFirstKey.length == 0)
912          currentRegionBoundariesInformation.metaFirstKey = null;
913        if (currentRegionBoundariesInformation.metaLastKey.length == 0)
914          currentRegionBoundariesInformation.metaLastKey = null;
915
916        // For a region to be correct, we need the META start key to be smaller or equal to the
917        // smallest start key from all the stores, and the start key from the next META entry to
918        // be bigger than the last key from all the current stores. First region start key is null;
919        // Last region end key is null; some regions can be empty and not have any store.
920
921        boolean valid = true;
922        // Checking start key.
923        if (
924          (currentRegionBoundariesInformation.storesFirstKey != null)
925            && (currentRegionBoundariesInformation.metaFirstKey != null)
926        ) {
927          valid = valid && comparator.compare(currentRegionBoundariesInformation.storesFirstKey,
928            currentRegionBoundariesInformation.metaFirstKey) >= 0;
929        }
930        // Checking stop key.
931        if (
932          (currentRegionBoundariesInformation.storesLastKey != null)
933            && (currentRegionBoundariesInformation.metaLastKey != null)
934        ) {
935          valid = valid && comparator.compare(currentRegionBoundariesInformation.storesLastKey,
936            currentRegionBoundariesInformation.metaLastKey) < 0;
937        }
938        if (!valid) {
939          errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, "Found issues with regions boundaries",
940            tablesInfo.get(regionInfo.getTable()));
941          LOG.warn("Region's boundaries not aligned between stores and META for:");
942          LOG.warn(Objects.toString(currentRegionBoundariesInformation));
943        }
944      }
945    } catch (IOException e) {
946      LOG.error(e.toString(), e);
947    }
948  }
949
950  /**
951   * Iterates through the list of all orphan/invalid regiondirs.
952   */
953  private void adoptHdfsOrphans(Collection<HbckRegionInfo> orphanHdfsDirs) throws IOException {
954    for (HbckRegionInfo hi : orphanHdfsDirs) {
955      LOG.info("Attempting to handle orphan hdfs dir: " + hi.getHdfsRegionDir());
956      adoptHdfsOrphan(hi);
957    }
958  }
959
960  /**
961   * Orphaned regions are regions without a .regioninfo file in them. We "adopt" these orphans by
962   * creating a new region, and moving the column families, recovered edits, WALs, into the new
963   * region dir. We determine the region startkey and endkeys by looking at all of the hfiles inside
964   * the column families to identify the min and max keys. The resulting region will likely violate
965   * table integrity but will be dealt with by merging overlapping regions.
966   */
967  @SuppressWarnings("deprecation")
968  private void adoptHdfsOrphan(HbckRegionInfo hi) throws IOException {
969    Path p = hi.getHdfsRegionDir();
970    FileSystem fs = p.getFileSystem(getConf());
971    FileStatus[] dirs = fs.listStatus(p);
972    if (dirs == null) {
973      LOG.warn("Attempt to adopt orphan hdfs region skipped because no files present in " + p
974        + ". This dir could probably be deleted.");
975      return;
976    }
977
978    TableName tableName = hi.getTableName();
979    HbckTableInfo tableInfo = tablesInfo.get(tableName);
980    Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!");
981    TableDescriptor template = tableInfo.getTableDescriptor();
982
983    // find min and max key values
984    Pair<byte[], byte[]> orphanRegionRange = null;
985    for (FileStatus cf : dirs) {
986      String cfName = cf.getPath().getName();
987      // TODO Figure out what the special dirs are
988      if (cfName.startsWith(".") || cfName.equals(HConstants.SPLIT_LOGDIR_NAME)) continue;
989
990      FileStatus[] hfiles = fs.listStatus(cf.getPath());
991      for (FileStatus hfile : hfiles) {
992        byte[] start, end;
993        HFile.Reader hf = null;
994        try {
995          hf = HFile.createReader(fs, hfile.getPath(), CacheConfig.DISABLED, true, getConf());
996          Optional<Cell> startKv = hf.getFirstKey();
997          start = CellUtil.cloneRow(startKv.get());
998          Optional<Cell> endKv = hf.getLastKey();
999          end = CellUtil.cloneRow(endKv.get());
1000        } catch (Exception ioe) {
1001          LOG.warn("Problem reading orphan file " + hfile + ", skipping");
1002          continue;
1003        } finally {
1004          if (hf != null) {
1005            hf.close();
1006          }
1007        }
1008
1009        // expand the range to include the range of all hfiles
1010        if (orphanRegionRange == null) {
1011          // first range
1012          orphanRegionRange = new Pair<>(start, end);
1013        } else {
1014          // TODO add test
1015
1016          // expand range only if the hfile is wider.
1017          if (Bytes.compareTo(orphanRegionRange.getFirst(), start) > 0) {
1018            orphanRegionRange.setFirst(start);
1019          }
1020          if (Bytes.compareTo(orphanRegionRange.getSecond(), end) < 0) {
1021            orphanRegionRange.setSecond(end);
1022          }
1023        }
1024      }
1025    }
1026    if (orphanRegionRange == null) {
1027      LOG.warn("No data in dir " + p + ", sidelining data");
1028      fixes++;
1029      sidelineRegionDir(fs, hi);
1030      return;
1031    }
1032    LOG.info("Min max keys are : [" + Bytes.toString(orphanRegionRange.getFirst()) + ", "
1033      + Bytes.toString(orphanRegionRange.getSecond()) + ")");
1034
1035    // create new region on hdfs. move data into place.
1036    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(template.getTableName())
1037      .setStartKey(orphanRegionRange.getFirst())
1038      .setEndKey(Bytes.add(orphanRegionRange.getSecond(), new byte[1])).build();
1039    LOG.info("Creating new region : " + regionInfo);
1040    HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), regionInfo, template);
1041    Path target = region.getRegionFileSystem().getRegionDir();
1042
1043    // rename all the data to new region
1044    mergeRegionDirs(target, hi);
1045    fixes++;
1046  }
1047
1048  /**
1049   * This method determines if there are table integrity errors in HDFS. If there are errors and the
1050   * appropriate "fix" options are enabled, the method will first correct orphan regions making them
1051   * into legit regiondirs, and then reload to merge potentially overlapping regions.
1052   * @return number of table integrity errors found
1053   */
1054  private int restoreHdfsIntegrity() throws IOException, InterruptedException {
1055    // Determine what's on HDFS
1056    LOG.info("Loading HBase regioninfo from HDFS...");
1057    loadHdfsRegionDirs(); // populating regioninfo table.
1058
1059    int errs = errors.getErrorList().size();
1060    // First time just get suggestions.
1061    tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
1062    checkHdfsIntegrity(false, false);
1063
1064    if (errors.getErrorList().size() == errs) {
1065      LOG.info("No integrity errors.  We are done with this phase. Glorious.");
1066      return 0;
1067    }
1068
1069    if (shouldFixHdfsOrphans() && orphanHdfsDirs.size() > 0) {
1070      adoptHdfsOrphans(orphanHdfsDirs);
1071      // TODO optimize by incrementally adding instead of reloading.
1072    }
1073
1074    // Make sure there are no holes now.
1075    if (shouldFixHdfsHoles()) {
1076      clearState(); // this also resets # fixes.
1077      loadHdfsRegionDirs();
1078      tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
1079      tablesInfo = checkHdfsIntegrity(shouldFixHdfsHoles(), false);
1080    }
1081
1082    // Now we fix overlaps
1083    if (shouldFixHdfsOverlaps()) {
1084      // second pass we fix overlaps.
1085      clearState(); // this also resets # fixes.
1086      loadHdfsRegionDirs();
1087      tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
1088      tablesInfo = checkHdfsIntegrity(false, shouldFixHdfsOverlaps());
1089    }
1090
1091    return errors.getErrorList().size();
1092  }
1093
1094  /**
1095   * Scan all the store file names to find any lingering reference files, which refer to some
1096   * none-exiting files. If "fix" option is enabled, any lingering reference file will be sidelined
1097   * if found.
1098   * <p>
1099   * Lingering reference file prevents a region from opening. It has to be fixed before a cluster
1100   * can start properly.
1101   */
1102  private void offlineReferenceFileRepair() throws IOException, InterruptedException {
1103    clearState();
1104    Configuration conf = getConf();
1105    Path hbaseRoot = CommonFSUtils.getRootDir(conf);
1106    FileSystem fs = hbaseRoot.getFileSystem(conf);
1107    LOG.info("Computing mapping of all store files");
1108    Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot,
1109      new FSUtils.ReferenceFileFilter(fs), executor, errors);
1110    errors.print("");
1111    LOG.info("Validating mapping using HDFS state");
1112    for (Path path : allFiles.values()) {
1113      Path referredToFile = StoreFileInfo.getReferredToFile(path);
1114      if (fs.exists(referredToFile)) continue; // good, expected
1115
1116      // Found a lingering reference file
1117      errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
1118        "Found lingering reference file " + path);
1119      if (!shouldFixReferenceFiles()) continue;
1120
1121      // Now, trying to fix it since requested
1122      boolean success = false;
1123      String pathStr = path.toString();
1124
1125      // A reference file path should be like
1126      // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/referred_file.region_name
1127      // Up 5 directories to get the root folder.
1128      // So the file will be sidelined to a similar folder structure.
1129      int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
1130      for (int i = 0; index > 0 && i < 5; i++) {
1131        index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1);
1132      }
1133      if (index > 0) {
1134        Path rootDir = getSidelineDir();
1135        Path dst = new Path(rootDir, pathStr.substring(index + 1));
1136        fs.mkdirs(dst.getParent());
1137        LOG.info("Trying to sideline reference file " + path + " to " + dst);
1138        setShouldRerun();
1139
1140        success = fs.rename(path, dst);
1141        debugLsr(dst);
1142
1143      }
1144      if (!success) {
1145        LOG.error("Failed to sideline reference file " + path);
1146      }
1147    }
1148  }
1149
1150  /**
1151   * Scan all the store file names to find any lingering HFileLink files, which refer to some
1152   * none-exiting files. If "fix" option is enabled, any lingering HFileLink file will be sidelined
1153   * if found.
1154   */
1155  private void offlineHLinkFileRepair() throws IOException, InterruptedException {
1156    Configuration conf = getConf();
1157    Path hbaseRoot = CommonFSUtils.getRootDir(conf);
1158    FileSystem fs = hbaseRoot.getFileSystem(conf);
1159    LOG.info("Computing mapping of all link files");
1160    Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot,
1161      new FSUtils.HFileLinkFilter(), executor, errors);
1162    errors.print("");
1163
1164    LOG.info("Validating mapping using HDFS state");
1165    for (Path path : allFiles.values()) {
1166      // building HFileLink object to gather locations
1167      HFileLink actualLink = HFileLink.buildFromHFileLinkPattern(conf, path);
1168      if (actualLink.exists(fs)) continue; // good, expected
1169
1170      // Found a lingering HFileLink
1171      errors.reportError(ERROR_CODE.LINGERING_HFILELINK, "Found lingering HFileLink " + path);
1172      if (!shouldFixHFileLinks()) continue;
1173
1174      // Now, trying to fix it since requested
1175      setShouldRerun();
1176
1177      // An HFileLink path should be like
1178      // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/linkedtable=linkedregionname-linkedhfilename
1179      // sidelineing will happen in the ${hbase.rootdir}/${sidelinedir} directory with the same
1180      // folder structure.
1181      boolean success = sidelineFile(fs, hbaseRoot, path);
1182
1183      if (!success) {
1184        LOG.error("Failed to sideline HFileLink file " + path);
1185      }
1186
1187      // An HFileLink backreference path should be like
1188      // ${hbase.rootdir}/archive/data/namespace/table_name/region_id/family_name/.links-linkedhfilename
1189      // sidelineing will happen in the ${hbase.rootdir}/${sidelinedir} directory with the same
1190      // folder structure.
1191      Path backRefPath = FileLink.getBackReferencesDir(
1192        HFileArchiveUtil.getStoreArchivePath(conf,
1193          HFileLink.getReferencedTableName(path.getName().toString()),
1194          HFileLink.getReferencedRegionName(path.getName().toString()), path.getParent().getName()),
1195        HFileLink.getReferencedHFileName(path.getName().toString()));
1196      success = sidelineFile(fs, hbaseRoot, backRefPath);
1197
1198      if (!success) {
1199        LOG.error("Failed to sideline HFileLink backreference file " + path);
1200      }
1201    }
1202  }
1203
1204  private boolean sidelineFile(FileSystem fs, Path hbaseRoot, Path path) throws IOException {
1205    URI uri = hbaseRoot.toUri().relativize(path.toUri());
1206    if (uri.isAbsolute()) return false;
1207    String relativePath = uri.getPath();
1208    Path rootDir = getSidelineDir();
1209    Path dst = new Path(rootDir, relativePath);
1210    boolean pathCreated = fs.mkdirs(dst.getParent());
1211    if (!pathCreated) {
1212      LOG.error("Failed to create path: " + dst.getParent());
1213      return false;
1214    }
1215    LOG.info("Trying to sideline file " + path + " to " + dst);
1216    return fs.rename(path, dst);
1217  }
1218
1219  /**
1220   * TODO -- need to add tests for this.
1221   */
1222  private void reportEmptyMetaCells() {
1223    errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: "
1224      + emptyRegionInfoQualifiers.size());
1225    if (details) {
1226      for (Result r : emptyRegionInfoQualifiers) {
1227        errors.print("  " + r);
1228      }
1229    }
1230  }
1231
1232  /**
1233   * TODO -- need to add tests for this.
1234   */
1235  private void reportTablesInFlux() {
1236    AtomicInteger numSkipped = new AtomicInteger(0);
1237    TableDescriptor[] allTables = getTables(numSkipped);
1238    errors.print("Number of Tables: " + allTables.length);
1239    if (details) {
1240      if (numSkipped.get() > 0) {
1241        errors.detail("Number of Tables in flux: " + numSkipped.get());
1242      }
1243      for (TableDescriptor td : allTables) {
1244        errors.detail("  Table: " + td.getTableName() + "\t" + (td.isReadOnly() ? "ro" : "rw")
1245          + "\t" + (td.isMetaRegion() ? "META" : "    ") + "\t" + " families: "
1246          + td.getColumnFamilyCount());
1247      }
1248    }
1249  }
1250
1251  public HbckErrorReporter getErrors() {
1252    return errors;
1253  }
1254
1255  /**
1256   * Populate hbi's from regionInfos loaded from file system.
1257   */
1258  private SortedMap<TableName, HbckTableInfo> loadHdfsRegionInfos()
1259    throws IOException, InterruptedException {
1260    tablesInfo.clear(); // regenerating the data
1261    // generate region split structure
1262    Collection<HbckRegionInfo> hbckRegionInfos = regionInfoMap.values();
1263
1264    // Parallelized read of .regioninfo files.
1265    List<WorkItemHdfsRegionInfo> hbis = new ArrayList<>(hbckRegionInfos.size());
1266    List<Future<Void>> hbiFutures;
1267
1268    for (HbckRegionInfo hbi : hbckRegionInfos) {
1269      WorkItemHdfsRegionInfo work = new WorkItemHdfsRegionInfo(hbi, this, errors);
1270      hbis.add(work);
1271    }
1272
1273    // Submit and wait for completion
1274    hbiFutures = executor.invokeAll(hbis);
1275
1276    for (int i = 0; i < hbiFutures.size(); i++) {
1277      WorkItemHdfsRegionInfo work = hbis.get(i);
1278      Future<Void> f = hbiFutures.get(i);
1279      try {
1280        f.get();
1281      } catch (ExecutionException e) {
1282        LOG.warn("Failed to read .regioninfo file for region " + work.hbi.getRegionNameAsString(),
1283          e.getCause());
1284      }
1285    }
1286
1287    Path hbaseRoot = CommonFSUtils.getRootDir(getConf());
1288    FileSystem fs = hbaseRoot.getFileSystem(getConf());
1289    // serialized table info gathering.
1290    for (HbckRegionInfo hbi : hbckRegionInfos) {
1291
1292      if (hbi.getHdfsHRI() == null) {
1293        // was an orphan
1294        continue;
1295      }
1296
1297      // get table name from hdfs, populate various HBaseFsck tables.
1298      TableName tableName = hbi.getTableName();
1299      if (tableName == null) {
1300        // There was an entry in hbase:meta not in the HDFS?
1301        LOG.warn("tableName was null for: " + hbi);
1302        continue;
1303      }
1304
1305      HbckTableInfo modTInfo = tablesInfo.get(tableName);
1306      if (modTInfo == null) {
1307        // only executed once per table.
1308        modTInfo = new HbckTableInfo(tableName, this);
1309        tablesInfo.put(tableName, modTInfo);
1310        try {
1311          TableDescriptor htd =
1312            FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
1313          modTInfo.htds.add(htd);
1314        } catch (IOException ioe) {
1315          if (!orphanTableDirs.containsKey(tableName)) {
1316            LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
1317            // should only report once for each table
1318            errors.reportError(ERROR_CODE.NO_TABLEINFO_FILE,
1319              "Unable to read .tableinfo from " + hbaseRoot + "/" + tableName);
1320            Set<String> columns = new HashSet<>();
1321            orphanTableDirs.put(tableName, getColumnFamilyList(columns, hbi));
1322          }
1323        }
1324      }
1325      if (!hbi.isSkipChecks()) {
1326        modTInfo.addRegionInfo(hbi);
1327      }
1328    }
1329
1330    loadTableInfosForTablesWithNoRegion();
1331    errors.print("");
1332
1333    return tablesInfo;
1334  }
1335
1336  /**
1337   * To get the column family list according to the column family dirs
1338   * @return a set of column families
1339   */
1340  private Set<String> getColumnFamilyList(Set<String> columns, HbckRegionInfo hbi)
1341    throws IOException {
1342    Path regionDir = hbi.getHdfsRegionDir();
1343    FileSystem fs = regionDir.getFileSystem(getConf());
1344    FileStatus[] subDirs = fs.listStatus(regionDir, new FSUtils.FamilyDirFilter(fs));
1345    for (FileStatus subdir : subDirs) {
1346      String columnfamily = subdir.getPath().getName();
1347      columns.add(columnfamily);
1348    }
1349    return columns;
1350  }
1351
1352  /**
1353   * To fabricate a .tableinfo file with following contents<br>
1354   * 1. the correct tablename <br>
1355   * 2. the correct colfamily list<br>
1356   * 3. the default properties for both {@link TableDescriptor} and
1357   * {@link ColumnFamilyDescriptor}<br>
1358   */
1359  private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName,
1360    Set<String> columns) throws IOException {
1361    if (columns == null || columns.isEmpty()) return false;
1362    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
1363    for (String columnfamimly : columns) {
1364      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnfamimly));
1365    }
1366    fstd.createTableDescriptor(builder.build(), true);
1367    return true;
1368  }
1369
1370  /**
1371   * To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta <br>
1372   */
1373  public void fixEmptyMetaCells() throws IOException {
1374    if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) {
1375      LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows.");
1376      for (Result region : emptyRegionInfoQualifiers) {
1377        deleteMetaRegion(region.getRow());
1378        errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL);
1379      }
1380      emptyRegionInfoQualifiers.clear();
1381    }
1382  }
1383
1384  /**
1385   * To fix orphan table by creating a .tableinfo file under tableDir <br>
1386   * 1. if TableInfo is cached, to recover the .tableinfo accordingly <br>
1387   * 2. else create a default .tableinfo file with following items<br>
1388   * &nbsp;2.1 the correct tablename <br>
1389   * &nbsp;2.2 the correct colfamily list<br>
1390   * &nbsp;2.3 the default properties for both {@link TableDescriptor} and
1391   * {@link ColumnFamilyDescriptor}<br>
1392   */
1393  public void fixOrphanTables() throws IOException {
1394    if (shouldFixTableOrphans() && !orphanTableDirs.isEmpty()) {
1395
1396      List<TableName> tmpList = new ArrayList<>(orphanTableDirs.keySet().size());
1397      tmpList.addAll(orphanTableDirs.keySet());
1398      TableDescriptor[] htds = getTableDescriptors(tmpList);
1399      Iterator<Entry<TableName, Set<String>>> iter = orphanTableDirs.entrySet().iterator();
1400      int j = 0;
1401      int numFailedCase = 0;
1402      FSTableDescriptors fstd = new FSTableDescriptors(getConf());
1403      while (iter.hasNext()) {
1404        Entry<TableName, Set<String>> entry = iter.next();
1405        TableName tableName = entry.getKey();
1406        LOG.info("Trying to fix orphan table error: " + tableName);
1407        if (j < htds.length) {
1408          if (tableName.equals(htds[j].getTableName())) {
1409            TableDescriptor htd = htds[j];
1410            LOG.info("fixing orphan table: " + tableName + " from cache");
1411            fstd.createTableDescriptor(htd, true);
1412            j++;
1413            iter.remove();
1414          }
1415        } else {
1416          if (fabricateTableInfo(fstd, tableName, entry.getValue())) {
1417            LOG.warn("fixing orphan table: " + tableName + " with a default .tableinfo file");
1418            LOG.warn(
1419              "Strongly recommend to modify the TableDescriptor if necessary for: " + tableName);
1420            iter.remove();
1421          } else {
1422            LOG.error("Unable to create default .tableinfo for " + tableName
1423              + " while missing column family information");
1424            numFailedCase++;
1425          }
1426        }
1427        fixes++;
1428      }
1429
1430      if (orphanTableDirs.isEmpty()) {
1431        // all orphanTableDirs are luckily recovered
1432        // re-run doFsck after recovering the .tableinfo file
1433        setShouldRerun();
1434        LOG.warn(
1435          "Strongly recommend to re-run manually hfsck after all orphanTableDirs being fixed");
1436      } else if (numFailedCase > 0) {
1437        LOG.error("Failed to fix " + numFailedCase + " OrphanTables with default .tableinfo files");
1438      }
1439
1440    }
1441    // cleanup the list
1442    orphanTableDirs.clear();
1443
1444  }
1445
1446  /**
1447   * Log an appropriate message about whether or not overlapping merges are computed in parallel.
1448   */
1449  private void logParallelMerge() {
1450    if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) {
1451      LOG.info("Handling overlap merges in parallel. set hbasefsck.overlap.merge.parallel to"
1452        + " false to run serially.");
1453    } else {
1454      LOG.info("Handling overlap merges serially.  set hbasefsck.overlap.merge.parallel to"
1455        + " true to run in parallel.");
1456    }
1457  }
1458
1459  private SortedMap<TableName, HbckTableInfo> checkHdfsIntegrity(boolean fixHoles,
1460    boolean fixOverlaps) throws IOException {
1461    LOG.info("Checking HBase region split map from HDFS data...");
1462    logParallelMerge();
1463    for (HbckTableInfo tInfo : tablesInfo.values()) {
1464      TableIntegrityErrorHandler handler;
1465      if (fixHoles || fixOverlaps) {
1466        handler = tInfo.new HDFSIntegrityFixer(tInfo, errors, getConf(), fixHoles, fixOverlaps);
1467      } else {
1468        handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
1469      }
1470      if (!tInfo.checkRegionChain(handler)) {
1471        // should dump info as well.
1472        errors.report("Found inconsistency in table " + tInfo.getName());
1473      }
1474    }
1475    return tablesInfo;
1476  }
1477
1478  Path getSidelineDir() throws IOException {
1479    if (sidelineDir == null) {
1480      Path hbaseDir = CommonFSUtils.getRootDir(getConf());
1481      Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME);
1482      sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-" + startMillis);
1483    }
1484    return sidelineDir;
1485  }
1486
1487  /**
1488   * Sideline a region dir (instead of deleting it)
1489   */
1490  Path sidelineRegionDir(FileSystem fs, HbckRegionInfo hi) throws IOException {
1491    return sidelineRegionDir(fs, null, hi);
1492  }
1493
1494  /**
1495   * Sideline a region dir (instead of deleting it)
1496   * @param parentDir if specified, the region will be sidelined to folder like
1497   *                  {@literal .../parentDir/<table name>/<region name>}. The purpose is to group
1498   *                  together similar regions sidelined, for example, those regions should be bulk
1499   *                  loaded back later on. If NULL, it is ignored.
1500   */
1501  Path sidelineRegionDir(FileSystem fs, String parentDir, HbckRegionInfo hi) throws IOException {
1502    TableName tableName = hi.getTableName();
1503    Path regionDir = hi.getHdfsRegionDir();
1504
1505    if (!fs.exists(regionDir)) {
1506      LOG.warn("No previous " + regionDir + " exists.  Continuing.");
1507      return null;
1508    }
1509
1510    Path rootDir = getSidelineDir();
1511    if (parentDir != null) {
1512      rootDir = new Path(rootDir, parentDir);
1513    }
1514    Path sidelineTableDir = CommonFSUtils.getTableDir(rootDir, tableName);
1515    Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName());
1516    fs.mkdirs(sidelineRegionDir);
1517    boolean success = false;
1518    FileStatus[] cfs = fs.listStatus(regionDir);
1519    if (cfs == null) {
1520      LOG.info("Region dir is empty: " + regionDir);
1521    } else {
1522      for (FileStatus cf : cfs) {
1523        Path src = cf.getPath();
1524        Path dst = new Path(sidelineRegionDir, src.getName());
1525        if (fs.isFile(src)) {
1526          // simple file
1527          success = fs.rename(src, dst);
1528          if (!success) {
1529            String msg = "Unable to rename file " + src + " to " + dst;
1530            LOG.error(msg);
1531            throw new IOException(msg);
1532          }
1533          continue;
1534        }
1535
1536        // is a directory.
1537        fs.mkdirs(dst);
1538
1539        LOG.info("Sidelining files from " + src + " into containing region " + dst);
1540        // FileSystem.rename is inconsistent with directories -- if the
1541        // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir,
1542        // it moves the src into the dst dir resulting in (foo/a/b). If
1543        // the dst does not exist, and the src a dir, src becomes dst. (foo/b)
1544        FileStatus[] hfiles = fs.listStatus(src);
1545        if (hfiles != null && hfiles.length > 0) {
1546          for (FileStatus hfile : hfiles) {
1547            success = fs.rename(hfile.getPath(), dst);
1548            if (!success) {
1549              String msg = "Unable to rename file " + src + " to " + dst;
1550              LOG.error(msg);
1551              throw new IOException(msg);
1552            }
1553          }
1554        }
1555        LOG.debug("Sideline directory contents:");
1556        debugLsr(sidelineRegionDir);
1557      }
1558    }
1559
1560    LOG.info("Removing old region dir: " + regionDir);
1561    success = fs.delete(regionDir, true);
1562    if (!success) {
1563      String msg = "Unable to delete dir " + regionDir;
1564      LOG.error(msg);
1565      throw new IOException(msg);
1566    }
1567    return sidelineRegionDir;
1568  }
1569
1570  /**
1571   * Load the list of disabled tables in ZK into local set.
1572   */
1573  private void loadTableStates() throws IOException {
1574    tableStates = MetaTableAccessor.getTableStates(connection);
1575    // Add hbase:meta so this tool keeps working. In hbase2, meta is always enabled though it
1576    // has no entry in the table states. HBCK doesn't work right w/ hbase2 but just do this in
1577    // meantime.
1578    this.tableStates.put(TableName.META_TABLE_NAME,
1579      new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED));
1580  }
1581
1582  /**
1583   * Check if the specified region's table is disabled.
1584   * @param tableName table to check status of
1585   */
1586  boolean isTableDisabled(TableName tableName) {
1587    return tableStates.containsKey(tableName)
1588      && tableStates.get(tableName).inStates(TableState.State.DISABLED, TableState.State.DISABLING);
1589  }
1590
1591  /**
1592   * Scan HDFS for all regions, recording their information into regionInfoMap
1593   */
1594  public void loadHdfsRegionDirs() throws IOException, InterruptedException {
1595    Path rootDir = CommonFSUtils.getRootDir(getConf());
1596    FileSystem fs = rootDir.getFileSystem(getConf());
1597
1598    // list all tables from HDFS
1599    List<FileStatus> tableDirs = Lists.newArrayList();
1600
1601    boolean foundVersionFile = fs.exists(new Path(rootDir, HConstants.VERSION_FILE_NAME));
1602
1603    List<Path> paths = FSUtils.getTableDirs(fs, rootDir);
1604    for (Path path : paths) {
1605      TableName tableName = CommonFSUtils.getTableName(path);
1606      if (
1607        (!checkMetaOnly && isTableIncluded(tableName))
1608          || tableName.equals(TableName.META_TABLE_NAME)
1609      ) {
1610        tableDirs.add(fs.getFileStatus(path));
1611      }
1612    }
1613
1614    // verify that version file exists
1615    if (!foundVersionFile) {
1616      errors.reportError(ERROR_CODE.NO_VERSION_FILE,
1617        "Version file does not exist in root dir " + rootDir);
1618      if (shouldFixVersionFile()) {
1619        LOG.info("Trying to create a new " + HConstants.VERSION_FILE_NAME + " file.");
1620        setShouldRerun();
1621        FSUtils.setVersion(fs, rootDir,
1622          getConf().getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000),
1623          getConf().getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
1624            HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
1625      }
1626    }
1627
1628    // Avoid multithreading at table-level because already multithreaded internally at
1629    // region-level. Additionally multithreading at table-level can lead to deadlock
1630    // if there are many tables in the cluster. Since there are a limited # of threads
1631    // in the executor's thread pool and if we multithread at the table-level by putting
1632    // WorkItemHdfsDir callables into the executor, then we will have some threads in the
1633    // executor tied up solely in waiting for the tables' region-level calls to complete.
1634    // If there are enough tables then there will be no actual threads in the pool left
1635    // for the region-level callables to be serviced.
1636    for (FileStatus tableDir : tableDirs) {
1637      LOG.debug("Loading region dirs from " + tableDir.getPath());
1638      WorkItemHdfsDir item = new WorkItemHdfsDir(fs, errors, tableDir);
1639      try {
1640        item.call();
1641      } catch (ExecutionException e) {
1642        LOG.warn("Could not completely load table dir " + tableDir.getPath(), e.getCause());
1643      }
1644    }
1645    errors.print("");
1646  }
1647
1648  /**
1649   * Record the location of the hbase:meta region as found in ZooKeeper.
1650   */
1651  private boolean recordMetaRegion() throws IOException {
1652    RegionLocations rl =
1653      connection.locateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, false, false);
1654    if (rl == null) {
1655      errors.reportError(ERROR_CODE.NULL_META_REGION, "META region was not found in ZooKeeper");
1656      return false;
1657    }
1658    for (HRegionLocation metaLocation : rl.getRegionLocations()) {
1659      // Check if Meta region is valid and existing
1660      if (metaLocation == null) {
1661        errors.reportError(ERROR_CODE.NULL_META_REGION, "META region location is null");
1662        return false;
1663      }
1664      if (metaLocation.getRegionInfo() == null) {
1665        errors.reportError(ERROR_CODE.NULL_META_REGION, "META location regionInfo is null");
1666        return false;
1667      }
1668      if (metaLocation.getHostname() == null) {
1669        errors.reportError(ERROR_CODE.NULL_META_REGION, "META location hostName is null");
1670        return false;
1671      }
1672      ServerName sn = metaLocation.getServerName();
1673      HbckRegionInfo.MetaEntry m = new HbckRegionInfo.MetaEntry(metaLocation.getRegion(), sn,
1674        EnvironmentEdgeManager.currentTime());
1675      HbckRegionInfo hbckRegionInfo = regionInfoMap.get(metaLocation.getRegion().getEncodedName());
1676      if (hbckRegionInfo == null) {
1677        regionInfoMap.put(metaLocation.getRegion().getEncodedName(), new HbckRegionInfo(m));
1678      } else {
1679        hbckRegionInfo.setMetaEntry(m);
1680      }
1681    }
1682    return true;
1683  }
1684
1685  private ZKWatcher createZooKeeperWatcher() throws IOException {
1686    return new ZKWatcher(getConf(), "hbase Fsck", new Abortable() {
1687      @Override
1688      public void abort(String why, Throwable e) {
1689        LOG.error(why, e);
1690        System.exit(1);
1691      }
1692
1693      @Override
1694      public boolean isAborted() {
1695        return false;
1696      }
1697
1698    });
1699  }
1700
1701  /**
1702   * Contacts each regionserver and fetches metadata about regions.
1703   * @param regionServerList - the list of region servers to connect to
1704   * @throws IOException if a remote or network exception occurs
1705   */
1706  void processRegionServers(Collection<ServerName> regionServerList)
1707    throws IOException, InterruptedException {
1708
1709    List<WorkItemRegion> workItems = new ArrayList<>(regionServerList.size());
1710    List<Future<Void>> workFutures;
1711
1712    // loop to contact each region server in parallel
1713    for (ServerName rsinfo : regionServerList) {
1714      workItems.add(new WorkItemRegion(this, rsinfo, errors, connection));
1715    }
1716
1717    workFutures = executor.invokeAll(workItems);
1718
1719    for (int i = 0; i < workFutures.size(); i++) {
1720      WorkItemRegion item = workItems.get(i);
1721      Future<Void> f = workFutures.get(i);
1722      try {
1723        f.get();
1724      } catch (ExecutionException e) {
1725        LOG.warn("Could not process regionserver {}", item.rsinfo.getAddress(), e.getCause());
1726      }
1727    }
1728  }
1729
1730  /**
1731   * Check consistency of all regions that have been found in previous phases.
1732   */
1733  private void checkAndFixConsistency() throws IOException, KeeperException, InterruptedException {
1734    // Divide the checks in two phases. One for default/primary replicas and another
1735    // for the non-primary ones. Keeps code cleaner this way.
1736
1737    List<CheckRegionConsistencyWorkItem> workItems = new ArrayList<>(regionInfoMap.size());
1738    for (java.util.Map.Entry<String, HbckRegionInfo> e : regionInfoMap.entrySet()) {
1739      if (e.getValue().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
1740        workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
1741      }
1742    }
1743    checkRegionConsistencyConcurrently(workItems);
1744
1745    boolean prevHdfsCheck = shouldCheckHdfs();
1746    setCheckHdfs(false); // replicas don't have any hdfs data
1747    // Run a pass over the replicas and fix any assignment issues that exist on the currently
1748    // deployed/undeployed replicas.
1749    List<CheckRegionConsistencyWorkItem> replicaWorkItems = new ArrayList<>(regionInfoMap.size());
1750    for (java.util.Map.Entry<String, HbckRegionInfo> e : regionInfoMap.entrySet()) {
1751      if (e.getValue().getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
1752        replicaWorkItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
1753      }
1754    }
1755    checkRegionConsistencyConcurrently(replicaWorkItems);
1756    setCheckHdfs(prevHdfsCheck);
1757
1758    // If some regions is skipped during checkRegionConsistencyConcurrently() phase, we might
1759    // not get accurate state of the hbase if continuing. The config here allows users to tune
1760    // the tolerance of number of skipped region.
1761    // TODO: evaluate the consequence to continue the hbck operation without config.
1762    int terminateThreshold = getConf().getInt("hbase.hbck.skipped.regions.limit", 0);
1763    int numOfSkippedRegions = skippedRegions.size();
1764    if (numOfSkippedRegions > 0 && numOfSkippedRegions > terminateThreshold) {
1765      throw new IOException(
1766        numOfSkippedRegions + " region(s) could not be checked or repaired.  See logs for detail.");
1767    }
1768
1769    if (shouldCheckHdfs()) {
1770      checkAndFixTableStates();
1771    }
1772  }
1773
1774  /**
1775   * Check consistency of all regions using multiple threads concurrently.
1776   */
1777  private void
1778    checkRegionConsistencyConcurrently(final List<CheckRegionConsistencyWorkItem> workItems)
1779      throws IOException, KeeperException, InterruptedException {
1780    if (workItems.isEmpty()) {
1781      return; // nothing to check
1782    }
1783
1784    List<Future<Void>> workFutures = executor.invokeAll(workItems);
1785    for (Future<Void> f : workFutures) {
1786      try {
1787        f.get();
1788      } catch (ExecutionException e1) {
1789        LOG.warn("Could not check region consistency ", e1.getCause());
1790        if (e1.getCause() instanceof IOException) {
1791          throw (IOException) e1.getCause();
1792        } else if (e1.getCause() instanceof KeeperException) {
1793          throw (KeeperException) e1.getCause();
1794        } else if (e1.getCause() instanceof InterruptedException) {
1795          throw (InterruptedException) e1.getCause();
1796        } else {
1797          throw new IOException(e1.getCause());
1798        }
1799      }
1800    }
1801  }
1802
1803  class CheckRegionConsistencyWorkItem implements Callable<Void> {
1804    private final String key;
1805    private final HbckRegionInfo hbi;
1806
1807    CheckRegionConsistencyWorkItem(String key, HbckRegionInfo hbi) {
1808      this.key = key;
1809      this.hbi = hbi;
1810    }
1811
1812    @Override
1813    public synchronized Void call() throws Exception {
1814      try {
1815        checkRegionConsistency(key, hbi);
1816      } catch (Exception e) {
1817        // If the region is non-META region, skip this region and send warning/error message; if
1818        // the region is META region, we should not continue.
1819        LOG.warn(
1820          "Unable to complete check or repair the region '" + hbi.getRegionNameAsString() + "'.",
1821          e);
1822        if (hbi.getHdfsHRI().isMetaRegion()) {
1823          throw e;
1824        }
1825        LOG.warn("Skip region '" + hbi.getRegionNameAsString() + "'");
1826        addSkippedRegion(hbi);
1827      }
1828      return null;
1829    }
1830  }
1831
1832  private void addSkippedRegion(final HbckRegionInfo hbi) {
1833    Set<String> skippedRegionNames = skippedRegions.get(hbi.getTableName());
1834    if (skippedRegionNames == null) {
1835      skippedRegionNames = new HashSet<>();
1836    }
1837    skippedRegionNames.add(hbi.getRegionNameAsString());
1838    skippedRegions.put(hbi.getTableName(), skippedRegionNames);
1839  }
1840
1841  /**
1842   * Check and fix table states, assumes full info available: - tableInfos - empty tables loaded
1843   */
1844  private void checkAndFixTableStates() throws IOException {
1845    // first check dangling states
1846    for (Entry<TableName, TableState> entry : tableStates.entrySet()) {
1847      TableName tableName = entry.getKey();
1848      TableState tableState = entry.getValue();
1849      HbckTableInfo tableInfo = tablesInfo.get(tableName);
1850      if (isTableIncluded(tableName) && !tableName.isSystemTable() && tableInfo == null) {
1851        if (fixMeta) {
1852          MetaTableAccessor.deleteTableState(connection, tableName);
1853          TableState state = MetaTableAccessor.getTableState(connection, tableName);
1854          if (state != null) {
1855            errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE,
1856              tableName + " unable to delete dangling table state " + tableState);
1857          }
1858        } else if (!checkMetaOnly) {
1859          // dangling table state in meta if checkMetaOnly is false. If checkMetaOnly is
1860          // true, tableInfo will be null as tablesInfo are not polulated for all tables from hdfs
1861          errors.reportError(ERROR_CODE.ORPHAN_TABLE_STATE,
1862            tableName + " has dangling table state " + tableState);
1863        }
1864      }
1865    }
1866    // check that all tables have states
1867    for (TableName tableName : tablesInfo.keySet()) {
1868      if (isTableIncluded(tableName) && !tableStates.containsKey(tableName)) {
1869        if (fixMeta) {
1870          MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED);
1871          TableState newState = MetaTableAccessor.getTableState(connection, tableName);
1872          if (newState == null) {
1873            errors.reportError(ERROR_CODE.NO_TABLE_STATE,
1874              "Unable to change state for table " + tableName + " in meta ");
1875          }
1876        } else {
1877          errors.reportError(ERROR_CODE.NO_TABLE_STATE, tableName + " has no state in meta ");
1878        }
1879      }
1880    }
1881  }
1882
1883  private void preCheckPermission() throws IOException {
1884    if (shouldIgnorePreCheckPermission()) {
1885      return;
1886    }
1887
1888    Path hbaseDir = CommonFSUtils.getRootDir(getConf());
1889    FileSystem fs = hbaseDir.getFileSystem(getConf());
1890    UserProvider userProvider = UserProvider.instantiate(getConf());
1891    UserGroupInformation ugi = userProvider.getCurrent().getUGI();
1892    FileStatus[] files = fs.listStatus(hbaseDir);
1893    for (FileStatus file : files) {
1894      try {
1895        fs.access(file.getPath(), FsAction.WRITE);
1896      } catch (AccessControlException ace) {
1897        LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
1898        errors.reportError(ERROR_CODE.WRONG_USAGE,
1899          "Current user " + ugi.getUserName() + " does not have write perms to " + file.getPath()
1900            + ". Please rerun hbck as hdfs user " + file.getOwner());
1901        throw ace;
1902      }
1903    }
1904  }
1905
1906  /**
1907   * Deletes region from meta table
1908   */
1909  private void deleteMetaRegion(HbckRegionInfo hi) throws IOException {
1910    deleteMetaRegion(hi.getMetaEntry().getRegionName());
1911  }
1912
1913  /**
1914   * Deletes region from meta table
1915   */
1916  private void deleteMetaRegion(byte[] metaKey) throws IOException {
1917    Delete d = new Delete(metaKey);
1918    meta.delete(d);
1919    LOG.info("Deleted " + Bytes.toString(metaKey) + " from META");
1920  }
1921
1922  /**
1923   * Reset the split parent region info in meta table
1924   */
1925  private void resetSplitParent(HbckRegionInfo hi) throws IOException {
1926    RowMutations mutations = new RowMutations(hi.getMetaEntry().getRegionName());
1927    Delete d = new Delete(hi.getMetaEntry().getRegionName());
1928    d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER);
1929    d.addColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
1930    mutations.add(d);
1931
1932    RegionInfo hri =
1933      RegionInfoBuilder.newBuilder(hi.getMetaEntry()).setOffline(false).setSplit(false).build();
1934    Put p = MetaTableAccessor.makePutFromRegionInfo(hri);
1935    mutations.add(p);
1936
1937    meta.mutateRow(mutations);
1938    LOG.info("Reset split parent " + hi.getMetaEntry().getRegionNameAsString() + " in META");
1939  }
1940
1941  /**
1942   * This backwards-compatibility wrapper for permanently offlining a region that should not be
1943   * alive. If the region server does not support the "offline" method, it will use the closest
1944   * unassign method instead. This will basically work until one attempts to disable or delete the
1945   * affected table. The problem has to do with in-memory only master state, so restarting the
1946   * HMaster or failing over to another should fix this.
1947   */
1948  void offline(byte[] regionName) throws IOException {
1949    String regionString = Bytes.toStringBinary(regionName);
1950    if (!rsSupportsOffline) {
1951      LOG.warn("Using unassign region " + regionString
1952        + " instead of using offline method, you should" + " restart HMaster after these repairs");
1953      admin.unassign(regionName, true);
1954      return;
1955    }
1956
1957    // first time we assume the rs's supports #offline.
1958    try {
1959      LOG.info("Offlining region " + regionString);
1960      admin.offline(regionName);
1961    } catch (IOException ioe) {
1962      String notFoundMsg =
1963        "java.lang.NoSuchMethodException: " + "org.apache.hadoop.hbase.master.HMaster.offline([B)";
1964      if (ioe.getMessage().contains(notFoundMsg)) {
1965        LOG.warn(
1966          "Using unassign region " + regionString + " instead of using offline method, you should"
1967            + " restart HMaster after these repairs");
1968        rsSupportsOffline = false; // in the future just use unassign
1969        admin.unassign(regionName, true);
1970        return;
1971      }
1972      throw ioe;
1973    }
1974  }
1975
1976  /**
1977   * Attempts to undeploy a region from a region server based in information in META. Any operations
1978   * that modify the file system should make sure that its corresponding region is not deployed to
1979   * prevent data races. A separate call is required to update the master in-memory region state
1980   * kept in the AssignementManager. Because disable uses this state instead of that found in META,
1981   * we can't seem to cleanly disable/delete tables that have been hbck fixed. When used on a
1982   * version of HBase that does not have the offline ipc call exposed on the master (&lt;0.90.5,
1983   * &lt;0.92.0) a master restart or failover may be required.
1984   */
1985  void closeRegion(HbckRegionInfo hi) throws IOException, InterruptedException {
1986    if (hi.getMetaEntry() == null && hi.getHdfsEntry() == null) {
1987      undeployRegions(hi);
1988      return;
1989    }
1990
1991    // get assignment info and hregioninfo from meta.
1992    Get get = new Get(hi.getRegionName());
1993    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1994    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1995    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
1996    // also get the locations of the replicas to close if the primary region is being closed
1997    if (hi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
1998      int numReplicas = admin.getTableDescriptor(hi.getTableName()).getRegionReplication();
1999      for (int i = 0; i < numReplicas; i++) {
2000        get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i));
2001        get.addColumn(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i));
2002      }
2003    }
2004    Result r = meta.get(get);
2005    RegionLocations rl = MetaTableAccessor.getRegionLocations(r);
2006    if (rl == null) {
2007      LOG.warn("Unable to close region " + hi.getRegionNameAsString()
2008        + " since meta does not have handle to reach it");
2009      return;
2010    }
2011    for (HRegionLocation h : rl.getRegionLocations()) {
2012      ServerName serverName = h.getServerName();
2013      if (serverName == null) {
2014        errors.reportError("Unable to close region " + hi.getRegionNameAsString()
2015          + " because meta does not " + "have handle to reach it.");
2016        continue;
2017      }
2018      RegionInfo hri = h.getRegionInfo();
2019      if (hri == null) {
2020        LOG.warn("Unable to close region " + hi.getRegionNameAsString()
2021          + " because hbase:meta had invalid or missing " + HConstants.CATALOG_FAMILY_STR + ":"
2022          + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " qualifier value.");
2023        continue;
2024      }
2025      // close the region -- close files and remove assignment
2026      HBaseFsckRepair.closeRegionSilentlyAndWait(connection, serverName, hri);
2027    }
2028  }
2029
2030  private void undeployRegions(HbckRegionInfo hi) throws IOException, InterruptedException {
2031    undeployRegionsForHbi(hi);
2032    // undeploy replicas of the region (but only if the method is invoked for the primary)
2033    if (hi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
2034      return;
2035    }
2036    int numReplicas = admin.getDescriptor(hi.getTableName()).getRegionReplication();
2037    for (int i = 1; i < numReplicas; i++) {
2038      if (hi.getPrimaryHRIForDeployedReplica() == null) continue;
2039      RegionInfo hri =
2040        RegionReplicaUtil.getRegionInfoForReplica(hi.getPrimaryHRIForDeployedReplica(), i);
2041      HbckRegionInfo h = regionInfoMap.get(hri.getEncodedName());
2042      if (h != null) {
2043        undeployRegionsForHbi(h);
2044        // set skip checks; we undeployed it, and we don't want to evaluate this anymore
2045        // in consistency checks
2046        h.setSkipChecks(true);
2047      }
2048    }
2049  }
2050
2051  private void undeployRegionsForHbi(HbckRegionInfo hi) throws IOException, InterruptedException {
2052    for (HbckRegionInfo.OnlineEntry rse : hi.getOnlineEntries()) {
2053      LOG.debug("Undeploy region " + rse.getRegionInfo() + " from " + rse.getServerName());
2054      try {
2055        HBaseFsckRepair.closeRegionSilentlyAndWait(connection, rse.getServerName(),
2056          rse.getRegionInfo());
2057        offline(rse.getRegionInfo().getRegionName());
2058      } catch (IOException ioe) {
2059        LOG.warn("Got exception when attempting to offline region "
2060          + Bytes.toString(rse.getRegionInfo().getRegionName()), ioe);
2061      }
2062    }
2063  }
2064
2065  private void tryAssignmentRepair(HbckRegionInfo hbi, String msg)
2066    throws IOException, KeeperException, InterruptedException {
2067    // If we are trying to fix the errors
2068    if (shouldFixAssignments()) {
2069      errors.print(msg);
2070      undeployRegions(hbi);
2071      setShouldRerun();
2072      RegionInfo hri = hbi.getHdfsHRI();
2073      if (hri == null) {
2074        hri = hbi.getMetaEntry();
2075      }
2076      HBaseFsckRepair.fixUnassigned(admin, hri);
2077      HBaseFsckRepair.waitUntilAssigned(admin, hri);
2078
2079      // also assign replicas if needed (do it only when this call operates on a primary replica)
2080      if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) return;
2081      int replicationCount = admin.getTableDescriptor(hri.getTable()).getRegionReplication();
2082      for (int i = 1; i < replicationCount; i++) {
2083        hri = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
2084        HbckRegionInfo h = regionInfoMap.get(hri.getEncodedName());
2085        if (h != null) {
2086          undeployRegions(h);
2087          // set skip checks; we undeploy & deploy it; we don't want to evaluate this hbi anymore
2088          // in consistency checks
2089          h.setSkipChecks(true);
2090        }
2091        HBaseFsckRepair.fixUnassigned(admin, hri);
2092        HBaseFsckRepair.waitUntilAssigned(admin, hri);
2093      }
2094
2095    }
2096  }
2097
2098  /**
2099   * Check a single region for consistency and correct deployment.
2100   */
2101  private void checkRegionConsistency(final String key, final HbckRegionInfo hbi)
2102    throws IOException, KeeperException, InterruptedException {
2103
2104    if (hbi.isSkipChecks()) return;
2105    String descriptiveName = hbi.toString();
2106    boolean inMeta = hbi.getMetaEntry() != null;
2107    // In case not checking HDFS, assume the region is on HDFS
2108    boolean inHdfs = !shouldCheckHdfs() || hbi.getHdfsRegionDir() != null;
2109    boolean hasMetaAssignment = inMeta && hbi.getMetaEntry().regionServer != null;
2110    boolean isDeployed = !hbi.getDeployedOn().isEmpty();
2111    boolean isMultiplyDeployed = hbi.getDeployedOn().size() > 1;
2112    boolean deploymentMatchesMeta = hasMetaAssignment && isDeployed && !isMultiplyDeployed
2113      && hbi.getMetaEntry().regionServer.equals(hbi.getDeployedOn().get(0));
2114    boolean splitParent = inMeta && hbi.getMetaEntry().isSplit() && hbi.getMetaEntry().isOffline();
2115    boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.getMetaEntry().getTable());
2116    boolean recentlyModified =
2117      inHdfs && hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTime();
2118
2119    // ========== First the healthy cases =============
2120    if (hbi.containsOnlyHdfsEdits()) {
2121      return;
2122    }
2123    if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) {
2124      return;
2125    } else if (inMeta && inHdfs && !shouldBeDeployed && !isDeployed) {
2126      LOG.info("Region " + descriptiveName + " is in META, and in a disabled "
2127        + "tabled that is not deployed");
2128      return;
2129    } else if (recentlyModified) {
2130      LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
2131      return;
2132    }
2133    // ========== Cases where the region is not in hbase:meta =============
2134    else if (!inMeta && !inHdfs && !isDeployed) {
2135      // We shouldn't have record of this region at all then!
2136      assert false : "Entry for region with no data";
2137    } else if (!inMeta && !inHdfs && isDeployed) {
2138      errors.reportError(ERROR_CODE.NOT_IN_META_HDFS,
2139        "Region " + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but "
2140          + "deployed on " + Joiner.on(", ").join(hbi.getDeployedOn()));
2141      if (shouldFixAssignments()) {
2142        undeployRegions(hbi);
2143      }
2144
2145    } else if (!inMeta && inHdfs && !isDeployed) {
2146      if (hbi.isMerged()) {
2147        // This region has already been merged, the remaining hdfs file will be
2148        // cleaned by CatalogJanitor later
2149        hbi.setSkipChecks(true);
2150        LOG.info("Region " + descriptiveName
2151          + " got merge recently, its file(s) will be cleaned by CatalogJanitor later");
2152        return;
2153      }
2154      errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region " + descriptiveName
2155        + " on HDFS, but not listed in hbase:meta " + "or deployed on any region server");
2156      // restore region consistency of an adopted orphan
2157      if (shouldFixMeta()) {
2158        if (!hbi.isHdfsRegioninfoPresent()) {
2159          LOG.error("Region " + hbi.getHdfsHRI() + " could have been repaired"
2160            + " in table integrity repair phase if -fixHdfsOrphans was" + " used.");
2161          return;
2162        }
2163
2164        RegionInfo hri = hbi.getHdfsHRI();
2165        HbckTableInfo tableInfo = tablesInfo.get(hri.getTable());
2166
2167        for (RegionInfo region : tableInfo.getRegionsFromMeta(this.regionInfoMap)) {
2168          if (
2169            Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0
2170              && (region.getEndKey().length == 0
2171                || Bytes.compareTo(region.getEndKey(), hri.getEndKey()) >= 0)
2172              && Bytes.compareTo(region.getStartKey(), hri.getEndKey()) <= 0
2173          ) {
2174            if (region.isSplit() || region.isOffline()) continue;
2175            Path regionDir = hbi.getHdfsRegionDir();
2176            FileSystem fs = regionDir.getFileSystem(getConf());
2177            List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDir);
2178            for (Path familyDir : familyDirs) {
2179              List<Path> referenceFilePaths = FSUtils.getReferenceFilePaths(fs, familyDir);
2180              for (Path referenceFilePath : referenceFilePaths) {
2181                Path parentRegionDir =
2182                  StoreFileInfo.getReferredToFile(referenceFilePath).getParent().getParent();
2183                if (parentRegionDir.toString().endsWith(region.getEncodedName())) {
2184                  LOG.warn(hri + " start and stop keys are in the range of " + region
2185                    + ". The region might not be cleaned up from hdfs when region " + region
2186                    + " split failed. Hence deleting from hdfs.");
2187                  HRegionFileSystem.deleteRegionFromFileSystem(getConf(), fs, regionDir.getParent(),
2188                    hri);
2189                  return;
2190                }
2191              }
2192            }
2193          }
2194        }
2195        LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
2196        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
2197        HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
2198          admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(),
2199          numReplicas);
2200
2201        tryAssignmentRepair(hbi, "Trying to reassign region...");
2202      }
2203
2204    } else if (!inMeta && inHdfs && isDeployed) {
2205      errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
2206        + " not in META, but deployed on " + Joiner.on(", ").join(hbi.getDeployedOn()));
2207      debugLsr(hbi.getHdfsRegionDir());
2208      if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
2209        // for replicas, this means that we should undeploy the region (we would have
2210        // gone over the primaries and fixed meta holes in first phase under
2211        // checkAndFixConsistency; we shouldn't get the condition !inMeta at
2212        // this stage unless unwanted replica)
2213        if (shouldFixAssignments()) {
2214          undeployRegionsForHbi(hbi);
2215        }
2216      }
2217      if (shouldFixMeta() && hbi.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
2218        if (!hbi.isHdfsRegioninfoPresent()) {
2219          LOG.error("This should have been repaired in table integrity repair phase");
2220          return;
2221        }
2222
2223        LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
2224        int numReplicas = admin.getTableDescriptor(hbi.getTableName()).getRegionReplication();
2225        HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(),
2226          admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(),
2227          numReplicas);
2228        tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2229      }
2230
2231      // ========== Cases where the region is in hbase:meta =============
2232    } else if (inMeta && inHdfs && !isDeployed && splitParent) {
2233      // check whether this is an actual error, or just transient state where parent
2234      // is not cleaned
2235      if (hbi.getMetaEntry().splitA != null && hbi.getMetaEntry().splitB != null) {
2236        // check that split daughters are there
2237        HbckRegionInfo infoA = this.regionInfoMap.get(hbi.getMetaEntry().splitA.getEncodedName());
2238        HbckRegionInfo infoB = this.regionInfoMap.get(hbi.getMetaEntry().splitB.getEncodedName());
2239        if (infoA != null && infoB != null) {
2240          // we already processed or will process daughters. Move on, nothing to see here.
2241          hbi.setSkipChecks(true);
2242          return;
2243        }
2244      }
2245
2246      // For Replica region, we need to do a similar check. If replica is not split successfully,
2247      // error is going to be reported against primary daughter region.
2248      if (hbi.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
2249        LOG.info("Region " + descriptiveName + " is a split parent in META, in HDFS, "
2250          + "and not deployed on any region server. This may be transient.");
2251        hbi.setSkipChecks(true);
2252        return;
2253      }
2254
2255      errors.reportError(ERROR_CODE.LINGERING_SPLIT_PARENT,
2256        "Region " + descriptiveName + " is a split parent in META, in HDFS, "
2257          + "and not deployed on any region server. This could be transient, "
2258          + "consider to run the catalog janitor first!");
2259      if (shouldFixSplitParents()) {
2260        setShouldRerun();
2261        resetSplitParent(hbi);
2262      }
2263    } else if (inMeta && !inHdfs && !isDeployed) {
2264      errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region " + descriptiveName
2265        + " found in META, but not in HDFS " + "or deployed on any region server.");
2266      if (shouldFixMeta()) {
2267        deleteMetaRegion(hbi);
2268      }
2269    } else if (inMeta && !inHdfs && isDeployed) {
2270      errors.reportError(ERROR_CODE.NOT_IN_HDFS,
2271        "Region " + descriptiveName + " found in META, but not in HDFS, " + "and deployed on "
2272          + Joiner.on(", ").join(hbi.getDeployedOn()));
2273      // We treat HDFS as ground truth. Any information in meta is transient
2274      // and equivalent data can be regenerated. So, lets unassign and remove
2275      // these problems from META.
2276      if (shouldFixAssignments()) {
2277        errors.print("Trying to fix unassigned region...");
2278        undeployRegions(hbi);
2279      }
2280      if (shouldFixMeta()) {
2281        // wait for it to complete
2282        deleteMetaRegion(hbi);
2283      }
2284    } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) {
2285      errors.reportError(ERROR_CODE.NOT_DEPLOYED,
2286        "Region " + descriptiveName + " not deployed on any region server.");
2287      tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2288    } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) {
2289      errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
2290        "Region " + descriptiveName + " should not be deployed according "
2291          + "to META, but is deployed on " + Joiner.on(", ").join(hbi.getDeployedOn()));
2292      if (shouldFixAssignments()) {
2293        errors.print("Trying to close the region " + descriptiveName);
2294        setShouldRerun();
2295        HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn());
2296      }
2297    } else if (inMeta && inHdfs && isMultiplyDeployed) {
2298      errors.reportError(ERROR_CODE.MULTI_DEPLOYED,
2299        "Region " + descriptiveName + " is listed in hbase:meta on region server "
2300          + hbi.getMetaEntry().regionServer + " but is multiply assigned to region servers "
2301          + Joiner.on(", ").join(hbi.getDeployedOn()));
2302      // If we are trying to fix the errors
2303      if (shouldFixAssignments()) {
2304        errors.print("Trying to fix assignment error...");
2305        setShouldRerun();
2306        HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn());
2307      }
2308    } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
2309      errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META,
2310        "Region " + descriptiveName + " listed in hbase:meta on region server "
2311          + hbi.getMetaEntry().regionServer + " but found on region server "
2312          + hbi.getDeployedOn().get(0));
2313      // If we are trying to fix the errors
2314      if (shouldFixAssignments()) {
2315        errors.print("Trying to fix assignment error...");
2316        setShouldRerun();
2317        HBaseFsckRepair.fixMultiAssignment(connection, hbi.getMetaEntry(), hbi.getDeployedOn());
2318        HBaseFsckRepair.waitUntilAssigned(admin, hbi.getHdfsHRI());
2319      }
2320    } else {
2321      errors.reportError(ERROR_CODE.UNKNOWN,
2322        "Region " + descriptiveName + " is in an unforeseen state:" + " inMeta=" + inMeta
2323          + " inHdfs=" + inHdfs + " isDeployed=" + isDeployed + " isMultiplyDeployed="
2324          + isMultiplyDeployed + " deploymentMatchesMeta=" + deploymentMatchesMeta
2325          + " shouldBeDeployed=" + shouldBeDeployed);
2326    }
2327  }
2328
2329  /**
2330   * Checks tables integrity. Goes over all regions and scans the tables. Collects all the pieces
2331   * for each table and checks if there are missing, repeated or overlapping ones.
2332   */
2333  SortedMap<TableName, HbckTableInfo> checkIntegrity() throws IOException {
2334    tablesInfo = new TreeMap<>();
2335    LOG.debug("There are " + regionInfoMap.size() + " region info entries");
2336    for (HbckRegionInfo hbi : regionInfoMap.values()) {
2337      // Check only valid, working regions
2338      if (hbi.getMetaEntry() == null) {
2339        // this assumes that consistency check has run loadMetaEntry
2340        Path p = hbi.getHdfsRegionDir();
2341        if (p == null) {
2342          errors.report("No regioninfo in Meta or HDFS. " + hbi);
2343        }
2344
2345        // TODO test.
2346        continue;
2347      }
2348      if (hbi.getMetaEntry().regionServer == null) {
2349        errors.detail("Skipping region because no region server: " + hbi);
2350        continue;
2351      }
2352      if (hbi.getMetaEntry().isOffline()) {
2353        errors.detail("Skipping region because it is offline: " + hbi);
2354        continue;
2355      }
2356      if (hbi.containsOnlyHdfsEdits()) {
2357        errors.detail("Skipping region because it only contains edits" + hbi);
2358        continue;
2359      }
2360
2361      // Missing regionDir or over-deployment is checked elsewhere. Include
2362      // these cases in modTInfo, so we can evaluate those regions as part of
2363      // the region chain in META
2364      // if (hbi.foundRegionDir == null) continue;
2365      // if (hbi.deployedOn.size() != 1) continue;
2366      if (hbi.getDeployedOn().isEmpty()) {
2367        continue;
2368      }
2369
2370      // We should be safe here
2371      TableName tableName = hbi.getMetaEntry().getTable();
2372      HbckTableInfo modTInfo = tablesInfo.get(tableName);
2373      if (modTInfo == null) {
2374        modTInfo = new HbckTableInfo(tableName, this);
2375      }
2376      for (ServerName server : hbi.getDeployedOn()) {
2377        modTInfo.addServer(server);
2378      }
2379
2380      if (!hbi.isSkipChecks()) {
2381        modTInfo.addRegionInfo(hbi);
2382      }
2383
2384      tablesInfo.put(tableName, modTInfo);
2385    }
2386
2387    loadTableInfosForTablesWithNoRegion();
2388
2389    logParallelMerge();
2390    for (HbckTableInfo tInfo : tablesInfo.values()) {
2391      TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
2392      if (!tInfo.checkRegionChain(handler)) {
2393        errors.report("Found inconsistency in table " + tInfo.getName());
2394      }
2395    }
2396    return tablesInfo;
2397  }
2398
2399  /**
2400   * Loads table info's for tables that may not have been included, since there are no regions
2401   * reported for the table, but table dir is there in hdfs
2402   */
2403  private void loadTableInfosForTablesWithNoRegion() throws IOException {
2404    Map<String, TableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll();
2405    for (TableDescriptor htd : allTables.values()) {
2406      if (checkMetaOnly && !htd.isMetaTable()) {
2407        continue;
2408      }
2409
2410      TableName tableName = htd.getTableName();
2411      if (isTableIncluded(tableName) && !tablesInfo.containsKey(tableName)) {
2412        HbckTableInfo tableInfo = new HbckTableInfo(tableName, this);
2413        tableInfo.htds.add(htd);
2414        tablesInfo.put(htd.getTableName(), tableInfo);
2415      }
2416    }
2417  }
2418
2419  /**
2420   * Merge hdfs data by moving from contained HbckRegionInfo into targetRegionDir.
2421   * @return number of file move fixes done to merge regions.
2422   */
2423  public int mergeRegionDirs(Path targetRegionDir, HbckRegionInfo contained) throws IOException {
2424    int fileMoves = 0;
2425    String thread = Thread.currentThread().getName();
2426    LOG.debug("[" + thread + "] Contained region dir after close and pause");
2427    debugLsr(contained.getHdfsRegionDir());
2428
2429    // rename the contained into the container.
2430    FileSystem fs = targetRegionDir.getFileSystem(getConf());
2431    FileStatus[] dirs = null;
2432    try {
2433      dirs = fs.listStatus(contained.getHdfsRegionDir());
2434    } catch (FileNotFoundException fnfe) {
2435      // region we are attempting to merge in is not present! Since this is a merge, there is
2436      // no harm skipping this region if it does not exist.
2437      if (!fs.exists(contained.getHdfsRegionDir())) {
2438        LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2439          + " is missing. Assuming already sidelined or moved.");
2440      } else {
2441        sidelineRegionDir(fs, contained);
2442      }
2443      return fileMoves;
2444    }
2445
2446    if (dirs == null) {
2447      if (!fs.exists(contained.getHdfsRegionDir())) {
2448        LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2449          + " already sidelined.");
2450      } else {
2451        sidelineRegionDir(fs, contained);
2452      }
2453      return fileMoves;
2454    }
2455
2456    for (FileStatus cf : dirs) {
2457      Path src = cf.getPath();
2458      Path dst = new Path(targetRegionDir, src.getName());
2459
2460      if (src.getName().equals(HRegionFileSystem.REGION_INFO_FILE)) {
2461        // do not copy the old .regioninfo file.
2462        continue;
2463      }
2464
2465      if (src.getName().equals(HConstants.HREGION_OLDLOGDIR_NAME)) {
2466        // do not copy the .oldlogs files
2467        continue;
2468      }
2469
2470      LOG.info("[" + thread + "] Moving files from " + src + " into containing region " + dst);
2471      // FileSystem.rename is inconsistent with directories -- if the
2472      // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir,
2473      // it moves the src into the dst dir resulting in (foo/a/b). If
2474      // the dst does not exist, and the src a dir, src becomes dst. (foo/b)
2475      for (FileStatus hfile : fs.listStatus(src)) {
2476        boolean success = fs.rename(hfile.getPath(), dst);
2477        if (success) {
2478          fileMoves++;
2479        }
2480      }
2481      LOG.debug("[" + thread + "] Sideline directory contents:");
2482      debugLsr(targetRegionDir);
2483    }
2484
2485    // if all success.
2486    sidelineRegionDir(fs, contained);
2487    LOG.info("[" + thread + "] Sidelined region dir " + contained.getHdfsRegionDir() + " into "
2488      + getSidelineDir());
2489    debugLsr(contained.getHdfsRegionDir());
2490
2491    return fileMoves;
2492  }
2493
2494  static class WorkItemOverlapMerge implements Callable<Void> {
2495    private TableIntegrityErrorHandler handler;
2496    Collection<HbckRegionInfo> overlapgroup;
2497
2498    WorkItemOverlapMerge(Collection<HbckRegionInfo> overlapgroup,
2499      TableIntegrityErrorHandler handler) {
2500      this.handler = handler;
2501      this.overlapgroup = overlapgroup;
2502    }
2503
2504    @Override
2505    public Void call() throws Exception {
2506      handler.handleOverlapGroup(overlapgroup);
2507      return null;
2508    }
2509  };
2510
2511  /**
2512   * Return a list of user-space table names whose metadata have not been modified in the last few
2513   * milliseconds specified by timelag if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER,
2514   * STARTCODE_QUALIFIER, SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last
2515   * milliseconds specified by timelag, then the table is a candidate to be returned.
2516   * @return tables that have not been modified recently
2517   * @throws IOException if an error is encountered
2518   */
2519  TableDescriptor[] getTables(AtomicInteger numSkipped) {
2520    List<TableName> tableNames = new ArrayList<>();
2521    long now = EnvironmentEdgeManager.currentTime();
2522
2523    for (HbckRegionInfo hbi : regionInfoMap.values()) {
2524      HbckRegionInfo.MetaEntry info = hbi.getMetaEntry();
2525
2526      // if the start key is zero, then we have found the first region of a table.
2527      // pick only those tables that were not modified in the last few milliseconds.
2528      if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
2529        if (info.modTime + timelag < now) {
2530          tableNames.add(info.getTable());
2531        } else {
2532          numSkipped.incrementAndGet(); // one more in-flux table
2533        }
2534      }
2535    }
2536    return getTableDescriptors(tableNames);
2537  }
2538
2539  TableDescriptor[] getTableDescriptors(List<TableName> tableNames) {
2540    LOG.info("getTableDescriptors == tableNames => " + tableNames);
2541    try (Connection conn = ConnectionFactory.createConnection(getConf());
2542      Admin admin = conn.getAdmin()) {
2543      List<TableDescriptor> tds = admin.listTableDescriptors(tableNames);
2544      return tds.toArray(new TableDescriptor[tds.size()]);
2545    } catch (IOException e) {
2546      LOG.debug("Exception getting table descriptors", e);
2547    }
2548    return new TableDescriptor[0];
2549  }
2550
2551  /**
2552   * Gets the entry in regionInfo corresponding to the the given encoded region name. If the region
2553   * has not been seen yet, a new entry is added and returned.
2554   */
2555  private synchronized HbckRegionInfo getOrCreateInfo(String name) {
2556    HbckRegionInfo hbi = regionInfoMap.get(name);
2557    if (hbi == null) {
2558      hbi = new HbckRegionInfo(null);
2559      regionInfoMap.put(name, hbi);
2560    }
2561    return hbi;
2562  }
2563
2564  private void checkAndFixReplication() throws ReplicationException, IOException {
2565    ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors);
2566    checker.checkUnDeletedQueues();
2567
2568    if (checker.hasUnDeletedQueues() && this.fixReplication) {
2569      checker.fixUnDeletedQueues();
2570      setShouldRerun();
2571    }
2572  }
2573
2574  /**
2575   * Check values in regionInfo for hbase:meta Check if zero or more than one regions with
2576   * hbase:meta are found. If there are inconsistencies (i.e. zero or more than one regions pretend
2577   * to be holding the hbase:meta) try to fix that and report an error.
2578   * @throws IOException from HBaseFsckRepair functions
2579   */
2580  boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException {
2581    Map<Integer, HbckRegionInfo> metaRegions = new HashMap<>();
2582    for (HbckRegionInfo value : regionInfoMap.values()) {
2583      if (value.getMetaEntry() != null && value.getMetaEntry().isMetaRegion()) {
2584        metaRegions.put(value.getReplicaId(), value);
2585      }
2586    }
2587    int metaReplication =
2588      admin.getTableDescriptor(TableName.META_TABLE_NAME).getRegionReplication();
2589    boolean noProblem = true;
2590    // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas
2591    // Check the deployed servers. It should be exactly one server for each replica.
2592    for (int i = 0; i < metaReplication; i++) {
2593      HbckRegionInfo metaHbckRegionInfo = metaRegions.remove(i);
2594      List<ServerName> servers = new ArrayList<>();
2595      if (metaHbckRegionInfo != null) {
2596        servers = metaHbckRegionInfo.getDeployedOn();
2597      }
2598      if (servers.size() != 1) {
2599        noProblem = false;
2600        if (servers.isEmpty()) {
2601          assignMetaReplica(i);
2602        } else if (servers.size() > 1) {
2603          errors.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId "
2604            + metaHbckRegionInfo.getReplicaId() + " is found on more than one region.");
2605          if (shouldFixAssignments()) {
2606            errors.print("Trying to fix a problem with hbase:meta, replicaId "
2607              + metaHbckRegionInfo.getReplicaId() + "..");
2608            setShouldRerun();
2609            // try fix it (treat is a dupe assignment)
2610            HBaseFsckRepair.fixMultiAssignment(connection, metaHbckRegionInfo.getMetaEntry(),
2611              servers);
2612          }
2613        }
2614      }
2615    }
2616    // unassign whatever is remaining in metaRegions. They are excess replicas.
2617    for (Map.Entry<Integer, HbckRegionInfo> entry : metaRegions.entrySet()) {
2618      noProblem = false;
2619      errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
2620        "hbase:meta replicas are deployed in excess. Configured " + metaReplication + ", deployed "
2621          + metaRegions.size());
2622      if (shouldFixAssignments()) {
2623        errors.print(
2624          "Trying to undeploy excess replica, replicaId: " + entry.getKey() + " of hbase:meta..");
2625        setShouldRerun();
2626        unassignMetaReplica(entry.getValue());
2627      }
2628    }
2629    // if noProblem is false, rerun hbck with hopefully fixed META
2630    // if noProblem is true, no errors, so continue normally
2631    return noProblem;
2632  }
2633
2634  private void unassignMetaReplica(HbckRegionInfo hi)
2635    throws IOException, InterruptedException, KeeperException {
2636    undeployRegions(hi);
2637    ZKUtil.deleteNode(zkw,
2638      zkw.getZNodePaths().getZNodeForReplica(hi.getMetaEntry().getReplicaId()));
2639  }
2640
2641  private void assignMetaReplica(int replicaId)
2642    throws IOException, KeeperException, InterruptedException {
2643    errors.reportError(ERROR_CODE.NO_META_REGION,
2644      "hbase:meta, replicaId " + replicaId + " is not found on any region.");
2645    if (shouldFixAssignments()) {
2646      errors.print("Trying to fix a problem with hbase:meta..");
2647      setShouldRerun();
2648      // try to fix it (treat it as unassigned region)
2649      RegionInfo h = RegionReplicaUtil
2650        .getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
2651      HBaseFsckRepair.fixUnassigned(admin, h);
2652      HBaseFsckRepair.waitUntilAssigned(admin, h);
2653    }
2654  }
2655
2656  /**
2657   * Scan hbase:meta, adding all regions found to the regionInfo map.
2658   * @throws IOException if an error is encountered
2659   */
2660  boolean loadMetaEntries() throws IOException {
2661    MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
2662      int countRecord = 1;
2663
2664      // comparator to sort KeyValues with latest modtime
2665      final Comparator<Cell> comp = new Comparator<Cell>() {
2666        @Override
2667        public int compare(Cell k1, Cell k2) {
2668          return Long.compare(k1.getTimestamp(), k2.getTimestamp());
2669        }
2670      };
2671
2672      @Override
2673      public boolean visit(Result result) throws IOException {
2674        try {
2675
2676          // record the latest modification of this META record
2677          long ts = Collections.max(result.listCells(), comp).getTimestamp();
2678          RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
2679          if (rl == null) {
2680            emptyRegionInfoQualifiers.add(result);
2681            errors.reportError(ERROR_CODE.EMPTY_META_CELL,
2682              "Empty REGIONINFO_QUALIFIER found in hbase:meta");
2683            return true;
2684          }
2685          ServerName sn = null;
2686          if (
2687            rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID) == null
2688              || rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo() == null
2689          ) {
2690            emptyRegionInfoQualifiers.add(result);
2691            errors.reportError(ERROR_CODE.EMPTY_META_CELL,
2692              "Empty REGIONINFO_QUALIFIER found in hbase:meta");
2693            return true;
2694          }
2695          RegionInfo hri = rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegionInfo();
2696          if (!(isTableIncluded(hri.getTable()) || hri.isMetaRegion())) {
2697            return true;
2698          }
2699          PairOfSameType<RegionInfo> daughters = MetaTableAccessor.getDaughterRegions(result);
2700          for (HRegionLocation h : rl.getRegionLocations()) {
2701            if (h == null || h.getRegionInfo() == null) {
2702              continue;
2703            }
2704            sn = h.getServerName();
2705            hri = h.getRegionInfo();
2706
2707            HbckRegionInfo.MetaEntry m = null;
2708            if (hri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
2709              m = new HbckRegionInfo.MetaEntry(hri, sn, ts, daughters.getFirst(),
2710                daughters.getSecond());
2711            } else {
2712              m = new HbckRegionInfo.MetaEntry(hri, sn, ts, null, null);
2713            }
2714            HbckRegionInfo previous = regionInfoMap.get(hri.getEncodedName());
2715            if (previous == null) {
2716              regionInfoMap.put(hri.getEncodedName(), new HbckRegionInfo(m));
2717            } else if (previous.getMetaEntry() == null) {
2718              previous.setMetaEntry(m);
2719            } else {
2720              throw new IOException("Two entries in hbase:meta are same " + previous);
2721            }
2722          }
2723          List<RegionInfo> mergeParents = MetaTableAccessor.getMergeRegions(result.rawCells());
2724          if (mergeParents != null) {
2725            for (RegionInfo mergeRegion : mergeParents) {
2726              if (mergeRegion != null) {
2727                // This region is already being merged
2728                HbckRegionInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName());
2729                hbInfo.setMerged(true);
2730              }
2731            }
2732          }
2733
2734          // show proof of progress to the user, once for every 100 records.
2735          if (countRecord % 100 == 0) {
2736            errors.progress();
2737          }
2738          countRecord++;
2739          return true;
2740        } catch (RuntimeException e) {
2741          LOG.error("Result=" + result);
2742          throw e;
2743        }
2744      }
2745    };
2746    if (!checkMetaOnly) {
2747      // Scan hbase:meta to pick up user regions
2748      MetaTableAccessor.fullScanRegions(connection, visitor);
2749    }
2750
2751    errors.print("");
2752    return true;
2753  }
2754
2755  /**
2756   * Prints summary of all tables found on the system.
2757   */
2758  private void printTableSummary(SortedMap<TableName, HbckTableInfo> tablesInfo) {
2759    StringBuilder sb = new StringBuilder();
2760    int numOfSkippedRegions;
2761    errors.print("Summary:");
2762    for (HbckTableInfo tInfo : tablesInfo.values()) {
2763      numOfSkippedRegions = (skippedRegions.containsKey(tInfo.getName()))
2764        ? skippedRegions.get(tInfo.getName()).size()
2765        : 0;
2766
2767      if (errors.tableHasErrors(tInfo)) {
2768        errors.print("Table " + tInfo.getName() + " is inconsistent.");
2769      } else if (numOfSkippedRegions > 0) {
2770        errors.print("Table " + tInfo.getName() + " is okay (with " + numOfSkippedRegions
2771          + " skipped regions).");
2772      } else {
2773        errors.print("Table " + tInfo.getName() + " is okay.");
2774      }
2775      errors.print("    Number of regions: " + tInfo.getNumRegions());
2776      if (numOfSkippedRegions > 0) {
2777        Set<String> skippedRegionStrings = skippedRegions.get(tInfo.getName());
2778        System.out.println("    Number of skipped regions: " + numOfSkippedRegions);
2779        System.out.println("      List of skipped regions:");
2780        for (String sr : skippedRegionStrings) {
2781          System.out.println("        " + sr);
2782        }
2783      }
2784      sb.setLength(0); // clear out existing buffer, if any.
2785      sb.append("    Deployed on: ");
2786      for (ServerName server : tInfo.deployedOn) {
2787        sb.append(" " + server.toString());
2788      }
2789      errors.print(sb.toString());
2790    }
2791  }
2792
2793  static HbckErrorReporter getErrorReporter(final Configuration conf)
2794    throws ClassNotFoundException {
2795    Class<? extends HbckErrorReporter> reporter = conf.getClass("hbasefsck.errorreporter",
2796      PrintingErrorReporter.class, HbckErrorReporter.class);
2797    return ReflectionUtils.newInstance(reporter, conf);
2798  }
2799
2800  static class PrintingErrorReporter implements HbckErrorReporter {
2801    public int errorCount = 0;
2802    private int showProgress;
2803    // How frequently calls to progress() will create output
2804    private static final int progressThreshold = 100;
2805
2806    Set<HbckTableInfo> errorTables = new HashSet<>();
2807
2808    // for use by unit tests to verify which errors were discovered
2809    private ArrayList<ERROR_CODE> errorList = new ArrayList<>();
2810
2811    @Override
2812    public void clear() {
2813      errorTables.clear();
2814      errorList.clear();
2815      errorCount = 0;
2816    }
2817
2818    @Override
2819    public synchronized void reportError(ERROR_CODE errorCode, String message) {
2820      if (errorCode == ERROR_CODE.WRONG_USAGE) {
2821        System.err.println(message);
2822        return;
2823      }
2824
2825      errorList.add(errorCode);
2826      if (!summary) {
2827        System.out.println("ERROR: " + message);
2828      }
2829      errorCount++;
2830      showProgress = 0;
2831    }
2832
2833    @Override
2834    public synchronized void reportError(ERROR_CODE errorCode, String message,
2835      HbckTableInfo table) {
2836      errorTables.add(table);
2837      reportError(errorCode, message);
2838    }
2839
2840    @Override
2841    public synchronized void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table,
2842      HbckRegionInfo info) {
2843      errorTables.add(table);
2844      String reference = "(region " + info.getRegionNameAsString() + ")";
2845      reportError(errorCode, reference + " " + message);
2846    }
2847
2848    @Override
2849    public synchronized void reportError(ERROR_CODE errorCode, String message, HbckTableInfo table,
2850      HbckRegionInfo info1, HbckRegionInfo info2) {
2851      errorTables.add(table);
2852      String reference =
2853        "(regions " + info1.getRegionNameAsString() + " and " + info2.getRegionNameAsString() + ")";
2854      reportError(errorCode, reference + " " + message);
2855    }
2856
2857    @Override
2858    public synchronized void reportError(String message) {
2859      reportError(ERROR_CODE.UNKNOWN, message);
2860    }
2861
2862    /**
2863     * Report error information, but do not increment the error count. Intended for cases where the
2864     * actual error would have been reported previously.
2865     */
2866    @Override
2867    public synchronized void report(String message) {
2868      if (!summary) {
2869        System.out.println("ERROR: " + message);
2870      }
2871      showProgress = 0;
2872    }
2873
2874    @Override
2875    public synchronized int summarize() {
2876      System.out.println(Integer.toString(errorCount) + " inconsistencies detected.");
2877      if (errorCount == 0) {
2878        System.out.println("Status: OK");
2879        return 0;
2880      } else {
2881        System.out.println("Status: INCONSISTENT");
2882        return -1;
2883      }
2884    }
2885
2886    @Override
2887    public ArrayList<ERROR_CODE> getErrorList() {
2888      return errorList;
2889    }
2890
2891    @Override
2892    public synchronized void print(String message) {
2893      if (!summary) {
2894        System.out.println(message);
2895      }
2896    }
2897
2898    @Override
2899    public boolean tableHasErrors(HbckTableInfo table) {
2900      return errorTables.contains(table);
2901    }
2902
2903    @Override
2904    public void resetErrors() {
2905      errorCount = 0;
2906    }
2907
2908    @Override
2909    public synchronized void detail(String message) {
2910      if (details) {
2911        System.out.println(message);
2912      }
2913      showProgress = 0;
2914    }
2915
2916    @Override
2917    public synchronized void progress() {
2918      if (showProgress++ == progressThreshold) {
2919        if (!summary) {
2920          System.out.print(".");
2921        }
2922        showProgress = 0;
2923      }
2924    }
2925  }
2926
2927  /**
2928   * Contact a region server and get all information from it
2929   */
2930  static class WorkItemRegion implements Callable<Void> {
2931    private final HBaseFsck hbck;
2932    private final ServerName rsinfo;
2933    private final HbckErrorReporter errors;
2934    private final ClusterConnection connection;
2935
2936    WorkItemRegion(HBaseFsck hbck, ServerName info, HbckErrorReporter errors,
2937      ClusterConnection connection) {
2938      this.hbck = hbck;
2939      this.rsinfo = info;
2940      this.errors = errors;
2941      this.connection = connection;
2942    }
2943
2944    @Override
2945    public synchronized Void call() throws IOException {
2946      errors.progress();
2947      try {
2948        BlockingInterface server = connection.getAdmin(rsinfo);
2949
2950        // list all online regions from this region server
2951        List<RegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
2952        regions = filterRegions(regions);
2953
2954        if (details) {
2955          errors.detail(
2956            "RegionServer: " + rsinfo.getServerName() + " number of regions: " + regions.size());
2957          for (RegionInfo rinfo : regions) {
2958            errors.detail("  " + rinfo.getRegionNameAsString() + " id: " + rinfo.getRegionId()
2959              + " encoded_name: " + rinfo.getEncodedName() + " start: "
2960              + Bytes.toStringBinary(rinfo.getStartKey()) + " end: "
2961              + Bytes.toStringBinary(rinfo.getEndKey()));
2962          }
2963        }
2964
2965        // check to see if the existence of this region matches the region in META
2966
2967        for (RegionInfo r : regions) {
2968          HbckRegionInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
2969          hbi.addServer(r, rsinfo);
2970        }
2971      } catch (IOException e) { // unable to connect to the region server.
2972        errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE,
2973          "RegionServer: " + rsinfo.getServerName() + " Unable to fetch region information. " + e);
2974        throw e;
2975      }
2976      return null;
2977    }
2978
2979    private List<RegionInfo> filterRegions(List<RegionInfo> regions) {
2980      List<RegionInfo> ret = Lists.newArrayList();
2981      for (RegionInfo hri : regions) {
2982        if (hri.isMetaRegion() || (!hbck.checkMetaOnly && hbck.isTableIncluded(hri.getTable()))) {
2983          ret.add(hri);
2984        }
2985      }
2986      return ret;
2987    }
2988  }
2989
2990  /**
2991   * Contact hdfs and get all information about specified table directory into regioninfo list.
2992   */
2993  class WorkItemHdfsDir implements Callable<Void> {
2994    private FileStatus tableDir;
2995    private HbckErrorReporter errors;
2996    private FileSystem fs;
2997
2998    WorkItemHdfsDir(FileSystem fs, HbckErrorReporter errors, FileStatus status) {
2999      this.fs = fs;
3000      this.tableDir = status;
3001      this.errors = errors;
3002    }
3003
3004    @Override
3005    public synchronized Void call() throws InterruptedException, ExecutionException {
3006      final Vector<Exception> exceptions = new Vector<>();
3007
3008      try {
3009        final FileStatus[] regionDirs = fs.listStatus(tableDir.getPath());
3010        final List<Future<?>> futures = new ArrayList<>(regionDirs.length);
3011
3012        for (final FileStatus regionDir : regionDirs) {
3013          errors.progress();
3014          final String encodedName = regionDir.getPath().getName();
3015          // ignore directories that aren't hexadecimal
3016          if (!encodedName.toLowerCase(Locale.ROOT).matches("[0-9a-f]+")) {
3017            continue;
3018          }
3019
3020          if (!exceptions.isEmpty()) {
3021            break;
3022          }
3023
3024          futures.add(executor.submit(new Runnable() {
3025            @Override
3026            public void run() {
3027              try {
3028                LOG.debug("Loading region info from hdfs:" + regionDir.getPath());
3029
3030                Path regioninfoFile =
3031                  new Path(regionDir.getPath(), HRegionFileSystem.REGION_INFO_FILE);
3032                boolean regioninfoFileExists = fs.exists(regioninfoFile);
3033
3034                if (!regioninfoFileExists) {
3035                  // As tables become larger it is more and more likely that by the time you
3036                  // reach a given region that it will be gone due to region splits/merges.
3037                  if (!fs.exists(regionDir.getPath())) {
3038                    LOG.warn("By the time we tried to process this region dir it was already gone: "
3039                      + regionDir.getPath());
3040                    return;
3041                  }
3042                }
3043
3044                HbckRegionInfo hbi = HBaseFsck.this.getOrCreateInfo(encodedName);
3045                HbckRegionInfo.HdfsEntry he = new HbckRegionInfo.HdfsEntry();
3046                synchronized (hbi) {
3047                  if (hbi.getHdfsRegionDir() != null) {
3048                    errors
3049                      .print("Directory " + encodedName + " duplicate??" + hbi.getHdfsRegionDir());
3050                  }
3051
3052                  he.regionDir = regionDir.getPath();
3053                  he.regionDirModTime = regionDir.getModificationTime();
3054                  he.hdfsRegioninfoFilePresent = regioninfoFileExists;
3055                  // we add to orphan list when we attempt to read .regioninfo
3056
3057                  // Set a flag if this region contains only edits
3058                  // This is special case if a region is left after split
3059                  he.hdfsOnlyEdits = true;
3060                  FileStatus[] subDirs = fs.listStatus(regionDir.getPath());
3061                  Path ePath = WALSplitUtil.getRegionDirRecoveredEditsDir(regionDir.getPath());
3062                  for (FileStatus subDir : subDirs) {
3063                    errors.progress();
3064                    String sdName = subDir.getPath().getName();
3065                    if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) {
3066                      he.hdfsOnlyEdits = false;
3067                      break;
3068                    }
3069                  }
3070                  hbi.setHdfsEntry(he);
3071                }
3072              } catch (Exception e) {
3073                LOG.error("Could not load region dir", e);
3074                exceptions.add(e);
3075              }
3076            }
3077          }));
3078        }
3079
3080        // Ensure all pending tasks are complete (or that we run into an exception)
3081        for (Future<?> f : futures) {
3082          if (!exceptions.isEmpty()) {
3083            break;
3084          }
3085          try {
3086            f.get();
3087          } catch (ExecutionException e) {
3088            LOG.error("Unexpected exec exception!  Should've been caught already.  (Bug?)", e);
3089            // Shouldn't happen, we already logged/caught any exceptions in the Runnable
3090          }
3091          ;
3092        }
3093      } catch (IOException e) {
3094        LOG.error("Cannot execute WorkItemHdfsDir for " + tableDir, e);
3095        exceptions.add(e);
3096      } finally {
3097        if (!exceptions.isEmpty()) {
3098          errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "Table Directory: "
3099            + tableDir.getPath().getName() + " Unable to fetch all HDFS region information. ");
3100          // Just throw the first exception as an indication something bad happened
3101          // Don't need to propagate all the exceptions, we already logged them all anyway
3102          throw new ExecutionException("First exception in WorkItemHdfsDir",
3103            exceptions.firstElement());
3104        }
3105      }
3106      return null;
3107    }
3108  }
3109
3110  /**
3111   * Contact hdfs and get all information about specified table directory into regioninfo list.
3112   */
3113  static class WorkItemHdfsRegionInfo implements Callable<Void> {
3114    private HbckRegionInfo hbi;
3115    private HBaseFsck hbck;
3116    private HbckErrorReporter errors;
3117
3118    WorkItemHdfsRegionInfo(HbckRegionInfo hbi, HBaseFsck hbck, HbckErrorReporter errors) {
3119      this.hbi = hbi;
3120      this.hbck = hbck;
3121      this.errors = errors;
3122    }
3123
3124    @Override
3125    public synchronized Void call() throws IOException {
3126      // only load entries that haven't been loaded yet.
3127      if (hbi.getHdfsHRI() == null) {
3128        try {
3129          errors.progress();
3130          hbi.loadHdfsRegioninfo(hbck.getConf());
3131        } catch (IOException ioe) {
3132          String msg = "Orphan region in HDFS: Unable to load .regioninfo from table "
3133            + hbi.getTableName() + " in hdfs dir " + hbi.getHdfsRegionDir()
3134            + "!  It may be an invalid format or version file.  Treating as "
3135            + "an orphaned regiondir.";
3136          errors.reportError(ERROR_CODE.ORPHAN_HDFS_REGION, msg);
3137          try {
3138            hbck.debugLsr(hbi.getHdfsRegionDir());
3139          } catch (IOException ioe2) {
3140            LOG.error("Unable to read directory " + hbi.getHdfsRegionDir(), ioe2);
3141            throw ioe2;
3142          }
3143          hbck.orphanHdfsDirs.add(hbi);
3144          throw ioe;
3145        }
3146      }
3147      return null;
3148    }
3149  };
3150
3151  /**
3152   * Display the full report from fsck. This displays all live and dead region servers, and all
3153   * known regions.
3154   */
3155  public static void setDisplayFullReport() {
3156    details = true;
3157  }
3158
3159  public static boolean shouldDisplayFullReport() {
3160    return details;
3161  }
3162
3163  /**
3164   * Set exclusive mode.
3165   */
3166  public static void setForceExclusive() {
3167    forceExclusive = true;
3168  }
3169
3170  /**
3171   * Only one instance of hbck can modify HBase at a time.
3172   */
3173  public boolean isExclusive() {
3174    return fixAny || forceExclusive;
3175  }
3176
3177  /**
3178   * Set summary mode. Print only summary of the tables and status (OK or INCONSISTENT)
3179   */
3180  static void setSummary() {
3181    summary = true;
3182  }
3183
3184  /**
3185   * Set hbase:meta check mode. Print only info about hbase:meta table deployment/state
3186   */
3187  void setCheckMetaOnly() {
3188    checkMetaOnly = true;
3189  }
3190
3191  /**
3192   * Set region boundaries check mode.
3193   */
3194  void setRegionBoundariesCheck() {
3195    checkRegionBoundaries = true;
3196  }
3197
3198  /**
3199   * Set replication fix mode.
3200   */
3201  public void setFixReplication(boolean shouldFix) {
3202    fixReplication = shouldFix;
3203    fixAny |= shouldFix;
3204  }
3205
3206  public void setCleanReplicationBarrier(boolean shouldClean) {
3207    cleanReplicationBarrier = shouldClean;
3208  }
3209
3210  /**
3211   * Check if we should rerun fsck again. This checks if we've tried to fix something and we should
3212   * rerun fsck tool again. Display the full report from fsck. This displays all live and dead
3213   * region servers, and all known regions.
3214   */
3215  void setShouldRerun() {
3216    rerun = true;
3217  }
3218
3219  public boolean shouldRerun() {
3220    return rerun;
3221  }
3222
3223  /**
3224   * Fix inconsistencies found by fsck. This should try to fix errors (if any) found by fsck
3225   * utility.
3226   */
3227  public void setFixAssignments(boolean shouldFix) {
3228    fixAssignments = shouldFix;
3229    fixAny |= shouldFix;
3230  }
3231
3232  boolean shouldFixAssignments() {
3233    return fixAssignments;
3234  }
3235
3236  public void setFixMeta(boolean shouldFix) {
3237    fixMeta = shouldFix;
3238    fixAny |= shouldFix;
3239  }
3240
3241  boolean shouldFixMeta() {
3242    return fixMeta;
3243  }
3244
3245  public void setFixEmptyMetaCells(boolean shouldFix) {
3246    fixEmptyMetaCells = shouldFix;
3247    fixAny |= shouldFix;
3248  }
3249
3250  boolean shouldFixEmptyMetaCells() {
3251    return fixEmptyMetaCells;
3252  }
3253
3254  public void setCheckHdfs(boolean checking) {
3255    checkHdfs = checking;
3256  }
3257
3258  boolean shouldCheckHdfs() {
3259    return checkHdfs;
3260  }
3261
3262  public void setFixHdfsHoles(boolean shouldFix) {
3263    fixHdfsHoles = shouldFix;
3264    fixAny |= shouldFix;
3265  }
3266
3267  boolean shouldFixHdfsHoles() {
3268    return fixHdfsHoles;
3269  }
3270
3271  public void setFixTableOrphans(boolean shouldFix) {
3272    fixTableOrphans = shouldFix;
3273    fixAny |= shouldFix;
3274  }
3275
3276  boolean shouldFixTableOrphans() {
3277    return fixTableOrphans;
3278  }
3279
3280  public void setFixHdfsOverlaps(boolean shouldFix) {
3281    fixHdfsOverlaps = shouldFix;
3282    fixAny |= shouldFix;
3283  }
3284
3285  boolean shouldFixHdfsOverlaps() {
3286    return fixHdfsOverlaps;
3287  }
3288
3289  public void setFixHdfsOrphans(boolean shouldFix) {
3290    fixHdfsOrphans = shouldFix;
3291    fixAny |= shouldFix;
3292  }
3293
3294  boolean shouldFixHdfsOrphans() {
3295    return fixHdfsOrphans;
3296  }
3297
3298  public void setFixVersionFile(boolean shouldFix) {
3299    fixVersionFile = shouldFix;
3300    fixAny |= shouldFix;
3301  }
3302
3303  public boolean shouldFixVersionFile() {
3304    return fixVersionFile;
3305  }
3306
3307  public void setSidelineBigOverlaps(boolean sbo) {
3308    this.sidelineBigOverlaps = sbo;
3309  }
3310
3311  public boolean shouldSidelineBigOverlaps() {
3312    return sidelineBigOverlaps;
3313  }
3314
3315  public void setFixSplitParents(boolean shouldFix) {
3316    fixSplitParents = shouldFix;
3317    fixAny |= shouldFix;
3318  }
3319
3320  public void setRemoveParents(boolean shouldFix) {
3321    removeParents = shouldFix;
3322    fixAny |= shouldFix;
3323  }
3324
3325  boolean shouldFixSplitParents() {
3326    return fixSplitParents;
3327  }
3328
3329  boolean shouldRemoveParents() {
3330    return removeParents;
3331  }
3332
3333  public void setFixReferenceFiles(boolean shouldFix) {
3334    fixReferenceFiles = shouldFix;
3335    fixAny |= shouldFix;
3336  }
3337
3338  boolean shouldFixReferenceFiles() {
3339    return fixReferenceFiles;
3340  }
3341
3342  public void setFixHFileLinks(boolean shouldFix) {
3343    fixHFileLinks = shouldFix;
3344    fixAny |= shouldFix;
3345  }
3346
3347  boolean shouldFixHFileLinks() {
3348    return fixHFileLinks;
3349  }
3350
3351  public boolean shouldIgnorePreCheckPermission() {
3352    return !fixAny || ignorePreCheckPermission;
3353  }
3354
3355  public void setIgnorePreCheckPermission(boolean ignorePreCheckPermission) {
3356    this.ignorePreCheckPermission = ignorePreCheckPermission;
3357  }
3358
3359  /**
3360   * @param mm maximum number of regions to merge into a single region.
3361   */
3362  public void setMaxMerge(int mm) {
3363    this.maxMerge = mm;
3364  }
3365
3366  public int getMaxMerge() {
3367    return maxMerge;
3368  }
3369
3370  public void setMaxOverlapsToSideline(int mo) {
3371    this.maxOverlapsToSideline = mo;
3372  }
3373
3374  public int getMaxOverlapsToSideline() {
3375    return maxOverlapsToSideline;
3376  }
3377
3378  /**
3379   * Only check/fix tables specified by the list, Empty list means all tables are included.
3380   */
3381  boolean isTableIncluded(TableName table) {
3382    return (tablesIncluded.isEmpty()) || tablesIncluded.contains(table);
3383  }
3384
3385  public void includeTable(TableName table) {
3386    tablesIncluded.add(table);
3387  }
3388
3389  Set<TableName> getIncludedTables() {
3390    return new HashSet<>(tablesIncluded);
3391  }
3392
3393  /**
3394   * We are interested in only those tables that have not changed their state in hbase:meta during
3395   * the last few seconds specified by hbase.admin.fsck.timelag
3396   * @param seconds - the time in seconds
3397   */
3398  public void setTimeLag(long seconds) {
3399    timelag = seconds * 1000; // convert to milliseconds
3400  }
3401
3402  /**
3403   * @param sidelineDir - HDFS path to sideline data
3404   */
3405  public void setSidelineDir(String sidelineDir) {
3406    this.sidelineDir = new Path(sidelineDir);
3407  }
3408
3409  protected HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
3410    throws IOException {
3411    return new HFileCorruptionChecker(getConf(), executor, sidelineCorruptHFiles);
3412  }
3413
3414  public HFileCorruptionChecker getHFilecorruptionChecker() {
3415    return hfcc;
3416  }
3417
3418  public void setHFileCorruptionChecker(HFileCorruptionChecker hfcc) {
3419    this.hfcc = hfcc;
3420  }
3421
3422  public void setRetCode(int code) {
3423    this.retcode = code;
3424  }
3425
3426  public int getRetCode() {
3427    return retcode;
3428  }
3429
3430  protected HBaseFsck printUsageAndExit() {
3431    StringWriter sw = new StringWriter(2048);
3432    PrintWriter out = new PrintWriter(sw);
3433    out.println("");
3434    out.println("-----------------------------------------------------------------------");
3435    out.println("NOTE: As of HBase version 2.0, the hbck tool is significantly changed.");
3436    out.println("In general, all Read-Only options are supported and can be be used");
3437    out.println("safely. Most -fix/ -repair options are NOT supported. Please see usage");
3438    out.println("below for details on which options are not supported.");
3439    out.println("-----------------------------------------------------------------------");
3440    out.println("");
3441    out.println("Usage: fsck [opts] {only tables}");
3442    out.println(" where [opts] are:");
3443    out.println("   -help Display help options (this)");
3444    out.println("   -details Display full report of all regions.");
3445    out.println("   -timelag <timeInSeconds>  Process only regions that "
3446      + " have not experienced any metadata updates in the last " + " <timeInSeconds> seconds.");
3447    out.println("   -sleepBeforeRerun <timeInSeconds> Sleep this many seconds"
3448      + " before checking if the fix worked if run with -fix");
3449    out.println("   -summary Print only summary of the tables and status.");
3450    out.println("   -metaonly Only check the state of the hbase:meta table.");
3451    out.println("   -sidelineDir <hdfs://> HDFS path to backup existing meta.");
3452    out.println(
3453      "   -boundaries Verify that regions boundaries are the same between META and store files.");
3454    out.println("   -exclusive Abort if another hbck is exclusive or fixing.");
3455
3456    out.println("");
3457    out.println("  Datafile Repair options: (expert features, use with caution!)");
3458    out.println(
3459      "   -checkCorruptHFiles     Check all Hfiles by opening them to make sure they are valid");
3460    out.println(
3461      "   -sidelineCorruptHFiles  Quarantine corrupted HFiles.  implies -checkCorruptHFiles");
3462
3463    out.println("");
3464    out.println(" Replication options");
3465    out.println("   -fixReplication   Deletes replication queues for removed peers");
3466
3467    out.println("");
3468    out.println(
3469      "  Metadata Repair options supported as of version 2.0: (expert features, use with caution!)");
3470    out.println("   -fixVersionFile   Try to fix missing hbase.version file in hdfs.");
3471    out.println("   -fixReferenceFiles  Try to offline lingering reference store files");
3472    out.println("   -fixHFileLinks  Try to offline lingering HFileLinks");
3473    out.println("   -noHdfsChecking   Don't load/check region info from HDFS."
3474      + " Assumes hbase:meta region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap");
3475    out.println("   -ignorePreCheckPermission  ignore filesystem permission pre-check");
3476
3477    out.println("");
3478    out.println("NOTE: Following options are NOT supported as of HBase version 2.0+.");
3479    out.println("");
3480    out.println("  UNSUPPORTED Metadata Repair options: (expert features, use with caution!)");
3481    out.println(
3482      "   -fix              Try to fix region assignments.  This is for backwards compatibility");
3483    out.println("   -fixAssignments   Try to fix region assignments.  Replaces the old -fix");
3484    out.println(
3485      "   -fixMeta          Try to fix meta problems.  This assumes HDFS region info is good.");
3486    out.println("   -fixHdfsHoles     Try to fix region holes in hdfs.");
3487    out.println("   -fixHdfsOrphans   Try to fix region dirs with no .regioninfo file in hdfs");
3488    out.println(
3489      "   -fixTableOrphans  Try to fix table dirs with no .tableinfo file in hdfs (online mode only)");
3490    out.println("   -fixHdfsOverlaps  Try to fix region overlaps in hdfs.");
3491    out.println(
3492      "   -maxMerge <n>     When fixing region overlaps, allow at most <n> regions to merge. (n="
3493        + DEFAULT_MAX_MERGE + " by default)");
3494    out.println(
3495      "   -sidelineBigOverlaps  When fixing region overlaps, allow to sideline big overlaps");
3496    out.println(
3497      "   -maxOverlapsToSideline <n>  When fixing region overlaps, allow at most <n> regions to sideline per group. (n="
3498        + DEFAULT_OVERLAPS_TO_SIDELINE + " by default)");
3499    out.println("   -fixSplitParents  Try to force offline split parents to be online.");
3500    out.println(
3501      "   -removeParents    Try to offline and sideline lingering parents and keep daughter regions.");
3502    out.println("   -fixEmptyMetaCells  Try to fix hbase:meta entries not referencing any region"
3503      + " (empty REGIONINFO_QUALIFIER rows)");
3504
3505    out.println("");
3506    out.println("  UNSUPPORTED Metadata Repair shortcuts");
3507    out.println("   -repair           Shortcut for -fixAssignments -fixMeta -fixHdfsHoles "
3508      + "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps -fixReferenceFiles"
3509      + "-fixHFileLinks");
3510    out.println("   -repairHoles      Shortcut for -fixAssignments -fixMeta -fixHdfsHoles");
3511    out.println("");
3512    out.println(" Replication options");
3513    out.println("   -fixReplication   Deletes replication queues for removed peers");
3514    out.println("   -cleanReplicationBarrier [tableName] clean the replication barriers "
3515      + "of a specified table, tableName is required");
3516    out.flush();
3517    errors.reportError(ERROR_CODE.WRONG_USAGE, sw.toString());
3518
3519    setRetCode(-2);
3520    return this;
3521  }
3522
3523  /**
3524   * Main program
3525   */
3526  public static void main(String[] args) throws Exception {
3527    // create a fsck object
3528    Configuration conf = HBaseConfiguration.create();
3529    Path hbasedir = CommonFSUtils.getRootDir(conf);
3530    URI defaultFs = hbasedir.getFileSystem(conf).getUri();
3531    CommonFSUtils.setFsDefault(conf, new Path(defaultFs));
3532    int ret = ToolRunner.run(new HBaseFsckTool(conf), args);
3533    System.exit(ret);
3534  }
3535
3536  /**
3537   * This is a Tool wrapper that gathers -Dxxx=yyy configuration settings from the command line.
3538   */
3539  static class HBaseFsckTool extends Configured implements Tool {
3540    HBaseFsckTool(Configuration conf) {
3541      super(conf);
3542    }
3543
3544    @Override
3545    public int run(String[] args) throws Exception {
3546      HBaseFsck hbck = new HBaseFsck(getConf());
3547      hbck.exec(hbck.executor, args);
3548      hbck.close();
3549      return hbck.getRetCode();
3550    }
3551  };
3552
3553  public HBaseFsck exec(ExecutorService exec, String[] args)
3554    throws KeeperException, IOException, InterruptedException, ReplicationException {
3555    long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
3556
3557    boolean checkCorruptHFiles = false;
3558    boolean sidelineCorruptHFiles = false;
3559
3560    // Process command-line args.
3561    for (int i = 0; i < args.length; i++) {
3562      String cmd = args[i];
3563      if (cmd.equals("-help") || cmd.equals("-h")) {
3564        return printUsageAndExit();
3565      } else if (cmd.equals("-details")) {
3566        setDisplayFullReport();
3567      } else if (cmd.equals("-exclusive")) {
3568        setForceExclusive();
3569      } else if (cmd.equals("-timelag")) {
3570        if (i == args.length - 1) {
3571          errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value.");
3572          return printUsageAndExit();
3573        }
3574        try {
3575          long timelag = Long.parseLong(args[++i]);
3576          setTimeLag(timelag);
3577        } catch (NumberFormatException e) {
3578          errors.reportError(ERROR_CODE.WRONG_USAGE, "-timelag needs a numeric value.");
3579          return printUsageAndExit();
3580        }
3581      } else if (cmd.equals("-sleepBeforeRerun")) {
3582        if (i == args.length - 1) {
3583          errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -sleepBeforeRerun needs a value.");
3584          return printUsageAndExit();
3585        }
3586        try {
3587          sleepBeforeRerun = Long.parseLong(args[++i]);
3588        } catch (NumberFormatException e) {
3589          errors.reportError(ERROR_CODE.WRONG_USAGE, "-sleepBeforeRerun needs a numeric value.");
3590          return printUsageAndExit();
3591        }
3592      } else if (cmd.equals("-sidelineDir")) {
3593        if (i == args.length - 1) {
3594          errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -sidelineDir needs a value.");
3595          return printUsageAndExit();
3596        }
3597        setSidelineDir(args[++i]);
3598      } else if (cmd.equals("-fix")) {
3599        errors.reportError(ERROR_CODE.WRONG_USAGE,
3600          "This option is deprecated, please use  -fixAssignments instead.");
3601        setFixAssignments(true);
3602      } else if (cmd.equals("-fixAssignments")) {
3603        setFixAssignments(true);
3604      } else if (cmd.equals("-fixMeta")) {
3605        setFixMeta(true);
3606      } else if (cmd.equals("-noHdfsChecking")) {
3607        setCheckHdfs(false);
3608      } else if (cmd.equals("-fixHdfsHoles")) {
3609        setFixHdfsHoles(true);
3610      } else if (cmd.equals("-fixHdfsOrphans")) {
3611        setFixHdfsOrphans(true);
3612      } else if (cmd.equals("-fixTableOrphans")) {
3613        setFixTableOrphans(true);
3614      } else if (cmd.equals("-fixHdfsOverlaps")) {
3615        setFixHdfsOverlaps(true);
3616      } else if (cmd.equals("-fixVersionFile")) {
3617        setFixVersionFile(true);
3618      } else if (cmd.equals("-sidelineBigOverlaps")) {
3619        setSidelineBigOverlaps(true);
3620      } else if (cmd.equals("-fixSplitParents")) {
3621        setFixSplitParents(true);
3622      } else if (cmd.equals("-removeParents")) {
3623        setRemoveParents(true);
3624      } else if (cmd.equals("-ignorePreCheckPermission")) {
3625        setIgnorePreCheckPermission(true);
3626      } else if (cmd.equals("-checkCorruptHFiles")) {
3627        checkCorruptHFiles = true;
3628      } else if (cmd.equals("-sidelineCorruptHFiles")) {
3629        sidelineCorruptHFiles = true;
3630      } else if (cmd.equals("-fixReferenceFiles")) {
3631        setFixReferenceFiles(true);
3632      } else if (cmd.equals("-fixHFileLinks")) {
3633        setFixHFileLinks(true);
3634      } else if (cmd.equals("-fixEmptyMetaCells")) {
3635        setFixEmptyMetaCells(true);
3636      } else if (cmd.equals("-repair")) {
3637        // this attempts to merge overlapping hdfs regions, needs testing
3638        // under load
3639        setFixHdfsHoles(true);
3640        setFixHdfsOrphans(true);
3641        setFixMeta(true);
3642        setFixAssignments(true);
3643        setFixHdfsOverlaps(true);
3644        setFixVersionFile(true);
3645        setSidelineBigOverlaps(true);
3646        setFixSplitParents(false);
3647        setCheckHdfs(true);
3648        setFixReferenceFiles(true);
3649        setFixHFileLinks(true);
3650      } else if (cmd.equals("-repairHoles")) {
3651        // this will make all missing hdfs regions available but may lose data
3652        setFixHdfsHoles(true);
3653        setFixHdfsOrphans(false);
3654        setFixMeta(true);
3655        setFixAssignments(true);
3656        setFixHdfsOverlaps(false);
3657        setSidelineBigOverlaps(false);
3658        setFixSplitParents(false);
3659        setCheckHdfs(true);
3660      } else if (cmd.equals("-maxOverlapsToSideline")) {
3661        if (i == args.length - 1) {
3662          errors.reportError(ERROR_CODE.WRONG_USAGE,
3663            "-maxOverlapsToSideline needs a numeric value argument.");
3664          return printUsageAndExit();
3665        }
3666        try {
3667          int maxOverlapsToSideline = Integer.parseInt(args[++i]);
3668          setMaxOverlapsToSideline(maxOverlapsToSideline);
3669        } catch (NumberFormatException e) {
3670          errors.reportError(ERROR_CODE.WRONG_USAGE,
3671            "-maxOverlapsToSideline needs a numeric value argument.");
3672          return printUsageAndExit();
3673        }
3674      } else if (cmd.equals("-maxMerge")) {
3675        if (i == args.length - 1) {
3676          errors.reportError(ERROR_CODE.WRONG_USAGE, "-maxMerge needs a numeric value argument.");
3677          return printUsageAndExit();
3678        }
3679        try {
3680          int maxMerge = Integer.parseInt(args[++i]);
3681          setMaxMerge(maxMerge);
3682        } catch (NumberFormatException e) {
3683          errors.reportError(ERROR_CODE.WRONG_USAGE, "-maxMerge needs a numeric value argument.");
3684          return printUsageAndExit();
3685        }
3686      } else if (cmd.equals("-summary")) {
3687        setSummary();
3688      } else if (cmd.equals("-metaonly")) {
3689        setCheckMetaOnly();
3690      } else if (cmd.equals("-boundaries")) {
3691        setRegionBoundariesCheck();
3692      } else if (cmd.equals("-fixReplication")) {
3693        setFixReplication(true);
3694      } else if (cmd.equals("-cleanReplicationBarrier")) {
3695        setCleanReplicationBarrier(true);
3696        if (args[++i].startsWith("-")) {
3697          printUsageAndExit();
3698        }
3699        setCleanReplicationBarrierTable(args[i]);
3700      } else if (cmd.startsWith("-")) {
3701        errors.reportError(ERROR_CODE.WRONG_USAGE, "Unrecognized option:" + cmd);
3702        return printUsageAndExit();
3703      } else {
3704        includeTable(TableName.valueOf(cmd));
3705        errors.print("Allow checking/fixes for table: " + cmd);
3706      }
3707    }
3708
3709    errors.print("HBaseFsck command line options: " + StringUtils.join(args, " "));
3710
3711    // pre-check current user has FS write permission or not
3712    try {
3713      preCheckPermission();
3714    } catch (IOException ioe) {
3715      Runtime.getRuntime().exit(-1);
3716    }
3717
3718    // do the real work of hbck
3719    connect();
3720
3721    // after connecting to server above, we have server version
3722    // check if unsupported option is specified based on server version
3723    if (!isOptionsSupported(args)) {
3724      return printUsageAndExit();
3725    }
3726
3727    try {
3728      // if corrupt file mode is on, first fix them since they may be opened later
3729      if (checkCorruptHFiles || sidelineCorruptHFiles) {
3730        LOG.info("Checking all hfiles for corruption");
3731        HFileCorruptionChecker hfcc = createHFileCorruptionChecker(sidelineCorruptHFiles);
3732        setHFileCorruptionChecker(hfcc); // so we can get result
3733        Collection<TableName> tables = getIncludedTables();
3734        Collection<Path> tableDirs = new ArrayList<>();
3735        Path rootdir = CommonFSUtils.getRootDir(getConf());
3736        if (tables.size() > 0) {
3737          for (TableName t : tables) {
3738            tableDirs.add(CommonFSUtils.getTableDir(rootdir, t));
3739          }
3740        } else {
3741          tableDirs = FSUtils.getTableDirs(CommonFSUtils.getCurrentFileSystem(getConf()), rootdir);
3742        }
3743        hfcc.checkTables(tableDirs);
3744        hfcc.report(errors);
3745      }
3746
3747      // check and fix table integrity, region consistency.
3748      int code = onlineHbck();
3749      setRetCode(code);
3750      // If we have changed the HBase state it is better to run hbck again
3751      // to see if we haven't broken something else in the process.
3752      // We run it only once more because otherwise we can easily fall into
3753      // an infinite loop.
3754      if (shouldRerun()) {
3755        try {
3756          LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix...");
3757          Thread.sleep(sleepBeforeRerun);
3758        } catch (InterruptedException ie) {
3759          LOG.warn("Interrupted while sleeping");
3760          return this;
3761        }
3762        // Just report
3763        setFixAssignments(false);
3764        setFixMeta(false);
3765        setFixHdfsHoles(false);
3766        setFixHdfsOverlaps(false);
3767        setFixVersionFile(false);
3768        setFixTableOrphans(false);
3769        errors.resetErrors();
3770        code = onlineHbck();
3771        setRetCode(code);
3772      }
3773    } finally {
3774      IOUtils.closeQuietly(this, e -> LOG.warn("", e));
3775    }
3776    return this;
3777  }
3778
3779  private boolean isOptionsSupported(String[] args) {
3780    boolean result = true;
3781    String hbaseServerVersion = status.getHBaseVersion();
3782    if (VersionInfo.compareVersion("2.any.any", hbaseServerVersion) < 0) {
3783      // Process command-line args.
3784      for (String arg : args) {
3785        if (unsupportedOptionsInV2.contains(arg)) {
3786          errors.reportError(ERROR_CODE.UNSUPPORTED_OPTION,
3787            "option '" + arg + "' is not " + "supported!");
3788          result = false;
3789          break;
3790        }
3791      }
3792    }
3793    return result;
3794  }
3795
3796  public void setCleanReplicationBarrierTable(String cleanReplicationBarrierTable) {
3797    this.cleanReplicationBarrierTable = TableName.valueOf(cleanReplicationBarrierTable);
3798  }
3799
3800  public void cleanReplicationBarrier() throws IOException {
3801    if (!cleanReplicationBarrier || cleanReplicationBarrierTable == null) {
3802      return;
3803    }
3804    if (cleanReplicationBarrierTable.isSystemTable()) {
3805      errors.reportError(ERROR_CODE.INVALID_TABLE,
3806        "invalid table: " + cleanReplicationBarrierTable);
3807      return;
3808    }
3809
3810    boolean isGlobalScope = false;
3811    try {
3812      isGlobalScope = admin.getDescriptor(cleanReplicationBarrierTable).hasGlobalReplicationScope();
3813    } catch (TableNotFoundException e) {
3814      LOG.info("we may need to clean some erroneous data due to bugs");
3815    }
3816
3817    if (isGlobalScope) {
3818      errors.reportError(ERROR_CODE.INVALID_TABLE,
3819        "table's replication scope is global: " + cleanReplicationBarrierTable);
3820      return;
3821    }
3822    List<byte[]> regionNames = new ArrayList<>();
3823    Scan barrierScan = new Scan();
3824    barrierScan.setCaching(100);
3825    barrierScan.addFamily(HConstants.REPLICATION_BARRIER_FAMILY);
3826    barrierScan
3827      .withStartRow(MetaTableAccessor.getTableStartRowForMeta(cleanReplicationBarrierTable,
3828        MetaTableAccessor.QueryType.REGION))
3829      .withStopRow(MetaTableAccessor.getTableStopRowForMeta(cleanReplicationBarrierTable,
3830        MetaTableAccessor.QueryType.REGION));
3831    Result result;
3832    try (ResultScanner scanner = meta.getScanner(barrierScan)) {
3833      while ((result = scanner.next()) != null) {
3834        regionNames.add(result.getRow());
3835      }
3836    }
3837    if (regionNames.size() <= 0) {
3838      errors.reportError(ERROR_CODE.INVALID_TABLE,
3839        "there is no barriers of this table: " + cleanReplicationBarrierTable);
3840      return;
3841    }
3842    ReplicationQueueStorage queueStorage =
3843      ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
3844    List<ReplicationPeerDescription> peerDescriptions = admin.listReplicationPeers();
3845    if (peerDescriptions != null && peerDescriptions.size() > 0) {
3846      List<String> peers = peerDescriptions.stream()
3847        .filter(
3848          peerConfig -> peerConfig.getPeerConfig().needToReplicate(cleanReplicationBarrierTable))
3849        .map(peerConfig -> peerConfig.getPeerId()).collect(Collectors.toList());
3850      try {
3851        List<String> batch = new ArrayList<>();
3852        for (String peer : peers) {
3853          for (byte[] regionName : regionNames) {
3854            batch.add(RegionInfo.encodeRegionName(regionName));
3855            if (batch.size() % 100 == 0) {
3856              queueStorage.removeLastSequenceIds(peer, batch);
3857              batch.clear();
3858            }
3859          }
3860          if (batch.size() > 0) {
3861            queueStorage.removeLastSequenceIds(peer, batch);
3862            batch.clear();
3863          }
3864        }
3865      } catch (ReplicationException re) {
3866        throw new IOException(re);
3867      }
3868    }
3869    for (byte[] regionName : regionNames) {
3870      meta.delete(new Delete(regionName).addFamily(HConstants.REPLICATION_BARRIER_FAMILY));
3871    }
3872    setShouldRerun();
3873  }
3874
3875  /**
3876   * ls -r for debugging purposes
3877   */
3878  void debugLsr(Path p) throws IOException {
3879    debugLsr(getConf(), p, errors);
3880  }
3881
3882  /**
3883   * ls -r for debugging purposes
3884   */
3885  public static void debugLsr(Configuration conf, Path p) throws IOException {
3886    debugLsr(conf, p, new PrintingErrorReporter());
3887  }
3888
3889  /**
3890   * ls -r for debugging purposes
3891   */
3892  public static void debugLsr(Configuration conf, Path p, HbckErrorReporter errors)
3893    throws IOException {
3894    if (!LOG.isDebugEnabled() || p == null) {
3895      return;
3896    }
3897    FileSystem fs = p.getFileSystem(conf);
3898
3899    if (!fs.exists(p)) {
3900      // nothing
3901      return;
3902    }
3903    errors.print(p.toString());
3904
3905    if (fs.isFile(p)) {
3906      return;
3907    }
3908
3909    if (fs.getFileStatus(p).isDirectory()) {
3910      FileStatus[] fss = fs.listStatus(p);
3911      for (FileStatus status : fss) {
3912        debugLsr(conf, status.getPath(), errors);
3913      }
3914    }
3915  }
3916}