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