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