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