View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.util;
19  
20  import java.io.FileNotFoundException;
21  import java.io.IOException;
22  import java.io.PrintWriter;
23  import java.io.StringWriter;
24  import java.net.InetAddress;
25  import java.net.URI;
26  import java.util.ArrayList;
27  import java.util.Arrays;
28  import java.util.Collection;
29  import java.util.Collections;
30  import java.util.Comparator;
31  import java.util.HashMap;
32  import java.util.HashSet;
33  import java.util.Iterator;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.Map.Entry;
37  import java.util.Set;
38  import java.util.SortedMap;
39  import java.util.SortedSet;
40  import java.util.TreeMap;
41  import java.util.TreeSet;
42  import java.util.concurrent.Callable;
43  import java.util.concurrent.ConcurrentSkipListMap;
44  import java.util.concurrent.ExecutionException;
45  import java.util.concurrent.ExecutorService;
46  import java.util.concurrent.Future;
47  import java.util.concurrent.ScheduledThreadPoolExecutor;
48  import java.util.concurrent.atomic.AtomicInteger;
49  import java.util.concurrent.atomic.AtomicBoolean;
50  
51  import org.apache.commons.lang.StringUtils;
52  import org.apache.commons.logging.Log;
53  import org.apache.commons.logging.LogFactory;
54  import org.apache.hadoop.hbase.classification.InterfaceAudience;
55  import org.apache.hadoop.hbase.classification.InterfaceStability;
56  import org.apache.hadoop.conf.Configuration;
57  import org.apache.hadoop.conf.Configured;
58  import org.apache.hadoop.fs.FSDataOutputStream;
59  import org.apache.hadoop.fs.FileStatus;
60  import org.apache.hadoop.fs.FileSystem;
61  import org.apache.hadoop.fs.Path;
62  import org.apache.hadoop.fs.permission.FsAction;
63  import org.apache.hadoop.fs.permission.FsPermission;
64  import org.apache.hadoop.hbase.Abortable;
65  import org.apache.hadoop.hbase.Cell;
66  import org.apache.hadoop.hbase.ClusterStatus;
67  import org.apache.hadoop.hbase.HBaseConfiguration;
68  import org.apache.hadoop.hbase.HColumnDescriptor;
69  import org.apache.hadoop.hbase.HConstants;
70  import org.apache.hadoop.hbase.HRegionInfo;
71  import org.apache.hadoop.hbase.HRegionLocation;
72  import org.apache.hadoop.hbase.HTableDescriptor;
73  import org.apache.hadoop.hbase.KeyValue;
74  import org.apache.hadoop.hbase.MasterNotRunningException;
75  import org.apache.hadoop.hbase.ServerName;
76  import org.apache.hadoop.hbase.TableName;
77  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
78  import org.apache.hadoop.hbase.catalog.MetaEditor;
79  import org.apache.hadoop.hbase.client.Delete;
80  import org.apache.hadoop.hbase.client.Get;
81  import org.apache.hadoop.hbase.client.HBaseAdmin;
82  import org.apache.hadoop.hbase.client.HConnectable;
83  import org.apache.hadoop.hbase.client.HConnection;
84  import org.apache.hadoop.hbase.client.HConnectionManager;
85  import org.apache.hadoop.hbase.client.HTable;
86  import org.apache.hadoop.hbase.client.MetaScanner;
87  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
88  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
89  import org.apache.hadoop.hbase.client.Put;
90  import org.apache.hadoop.hbase.client.Result;
91  import org.apache.hadoop.hbase.client.RowMutations;
92  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
93  import org.apache.hadoop.hbase.io.hfile.HFile;
94  import org.apache.hadoop.hbase.master.MasterFileSystem;
95  import org.apache.hadoop.hbase.master.RegionState;
96  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
97  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
98  import org.apache.hadoop.hbase.regionserver.HRegion;
99  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
100 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
101 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
102 import org.apache.hadoop.hbase.security.UserProvider;
103 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
104 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
105 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
106 import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandler;
107 import org.apache.hadoop.hbase.util.hbck.TableIntegrityErrorHandlerImpl;
108 import org.apache.hadoop.hbase.util.hbck.TableLockChecker;
109 import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
110 import org.apache.hadoop.hbase.zookeeper.ZKTable;
111 import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
112 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
113 import org.apache.hadoop.hbase.security.AccessDeniedException;
114 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
115 import org.apache.hadoop.io.IOUtils;
116 import org.apache.hadoop.ipc.RemoteException;
117 import org.apache.hadoop.security.UserGroupInformation;
118 import org.apache.hadoop.util.ReflectionUtils;
119 import org.apache.hadoop.util.Tool;
120 import org.apache.hadoop.util.ToolRunner;
121 import org.apache.zookeeper.KeeperException;
122 
123 import com.google.common.base.Joiner;
124 import com.google.common.base.Preconditions;
125 import com.google.common.collect.ImmutableList;
126 import com.google.common.collect.Lists;
127 import com.google.common.collect.Multimap;
128 import com.google.common.collect.Ordering;
129 import com.google.common.collect.TreeMultimap;
130 import com.google.protobuf.ServiceException;
131 
132 /**
133  * HBaseFsck (hbck) is a tool for checking and repairing region consistency and
134  * table integrity problems in a corrupted HBase.
135  * <p>
136  * Region consistency checks verify that hbase:meta, region deployment on region
137  * servers and the state of data in HDFS (.regioninfo files) all are in
138  * accordance.
139  * <p>
140  * Table integrity checks verify that all possible row keys resolve to exactly
141  * one region of a table.  This means there are no individual degenerate
142  * or backwards regions; no holes between regions; and that there are no
143  * overlapping regions.
144  * <p>
145  * The general repair strategy works in two phases:
146  * <ol>
147  * <li> Repair Table Integrity on HDFS. (merge or fabricate regions)
148  * <li> Repair Region Consistency with hbase:meta and assignments
149  * </ol>
150  * <p>
151  * For table integrity repairs, the tables' region directories are scanned
152  * for .regioninfo files.  Each table's integrity is then verified.  If there
153  * are any orphan regions (regions with no .regioninfo files) or holes, new
154  * regions are fabricated.  Backwards regions are sidelined as well as empty
155  * degenerate (endkey==startkey) regions.  If there are any overlapping regions,
156  * a new region is created and all data is merged into the new region.
157  * <p>
158  * Table integrity repairs deal solely with HDFS and could potentially be done
159  * offline -- the hbase region servers or master do not need to be running.
160  * This phase can eventually be used to completely reconstruct the hbase:meta table in
161  * an offline fashion.
162  * <p>
163  * Region consistency requires three conditions -- 1) valid .regioninfo file
164  * present in an HDFS region dir,  2) valid row with .regioninfo data in META,
165  * and 3) a region is deployed only at the regionserver that was assigned to
166  * with proper state in the master.
167  * <p>
168  * Region consistency repairs require hbase to be online so that hbck can
169  * contact the HBase master and region servers.  The hbck#connect() method must
170  * first be called successfully.  Much of the region consistency information
171  * is transient and less risky to repair.
172  * <p>
173  * If hbck is run from the command line, there are a handful of arguments that
174  * can be used to limit the kinds of repairs hbck will do.  See the code in
175  * {@link #printUsageAndExit()} for more details.
176  */
177 @InterfaceAudience.Public
178 @InterfaceStability.Evolving
179 public class HBaseFsck extends Configured {
180   public static final long DEFAULT_TIME_LAG = 60000; // default value of 1 minute
181   public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000;
182   private static final int MAX_NUM_THREADS = 50; // #threads to contact regions
183   private static boolean rsSupportsOffline = true;
184   private static final int DEFAULT_OVERLAPS_TO_SIDELINE = 2;
185   private static final int DEFAULT_MAX_MERGE = 5;
186   private static final String TO_BE_LOADED = "to_be_loaded";
187   private static final String HBCK_LOCK_FILE = "hbase-hbck.lock";
188 
189 
190   /**********************
191    * Internal resources
192    **********************/
193   private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
194   private ClusterStatus status;
195   private HConnection connection;
196   private HBaseAdmin admin;
197   private HTable meta;
198   // threads to do ||izable tasks: retrieve data from regionservers, handle overlapping regions
199   protected ExecutorService executor;
200   private long startMillis = EnvironmentEdgeManager.currentTimeMillis();
201   private HFileCorruptionChecker hfcc;
202   private int retcode = 0;
203   private Path HBCK_LOCK_PATH;
204   private FSDataOutputStream hbckOutFd;
205   // This lock is to prevent cleanup of balancer resources twice between
206   // ShutdownHook and the main code. We cleanup only if the connect() is
207   // successful
208   private final AtomicBoolean hbckLockCleanup = new AtomicBoolean(false);
209 
210   /***********
211    * Options
212    ***********/
213   private static boolean details = false; // do we display the full report
214   private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
215   private boolean fixAssignments = false; // fix assignment errors?
216   private boolean fixMeta = false; // fix meta errors?
217   private boolean checkHdfs = true; // load and check fs consistency?
218   private boolean fixHdfsHoles = false; // fix fs holes?
219   private boolean fixHdfsOverlaps = false; // fix fs overlaps (risky)
220   private boolean fixHdfsOrphans = false; // fix fs holes (missing .regioninfo)
221   private boolean fixTableOrphans = false; // fix fs holes (missing .tableinfo)
222   private boolean fixVersionFile = false; // fix missing hbase.version file in hdfs
223   private boolean fixSplitParents = false; // fix lingering split parents
224   private boolean fixReferenceFiles = false; // fix lingering reference store file
225   private boolean fixEmptyMetaCells = false; // fix (remove) empty REGIONINFO_QUALIFIER rows
226   private boolean fixTableLocks = false; // fix table locks which are expired
227   private boolean fixTableZNodes = false; // fix table Znodes which are orphaned
228   private boolean fixAny = false; // Set to true if any of the fix is required.
229 
230   // limit checking/fixes to listed tables, if empty attempt to check/fix all
231   // hbase:meta are always checked
232   private Set<TableName> tablesIncluded = new HashSet<TableName>();
233   private int maxMerge = DEFAULT_MAX_MERGE; // maximum number of overlapping regions to merge
234   private int maxOverlapsToSideline = DEFAULT_OVERLAPS_TO_SIDELINE; // maximum number of overlapping regions to sideline
235   private boolean sidelineBigOverlaps = false; // sideline overlaps with >maxMerge regions
236   private Path sidelineDir = null;
237 
238   private boolean rerun = false; // if we tried to fix something, rerun hbck
239   private static boolean summary = false; // if we want to print less output
240   private boolean checkMetaOnly = false;
241   private boolean checkRegionBoundaries = false;
242   private boolean ignorePreCheckPermission = false; // if pre-check permission
243 
244   /*********
245    * State
246    *********/
247   final private ErrorReporter errors;
248   int fixes = 0;
249 
250   /**
251    * This map contains the state of all hbck items.  It maps from encoded region
252    * name to HbckInfo structure.  The information contained in HbckInfo is used
253    * to detect and correct consistency (hdfs/meta/deployment) problems.
254    */
255   private TreeMap<String, HbckInfo> regionInfoMap = new TreeMap<String, HbckInfo>();
256   private TreeSet<TableName> disabledTables =
257     new TreeSet<TableName>();
258   // Empty regioninfo qualifiers in hbase:meta
259   private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
260 
261   /**
262    * This map from Tablename -> TableInfo contains the structures necessary to
263    * detect table consistency problems (holes, dupes, overlaps).  It is sorted
264    * to prevent dupes.
265    *
266    * If tablesIncluded is empty, this map contains all tables.
267    * Otherwise, it contains only meta tables and tables in tablesIncluded,
268    * unless checkMetaOnly is specified, in which case, it contains only
269    * the meta table
270    */
271   private SortedMap<TableName, TableInfo> tablesInfo =
272       new ConcurrentSkipListMap<TableName, TableInfo>();
273 
274   /**
275    * When initially looking at HDFS, we attempt to find any orphaned data.
276    */
277   private List<HbckInfo> orphanHdfsDirs = Collections.synchronizedList(new ArrayList<HbckInfo>());
278 
279   private Map<TableName, Set<String>> orphanTableDirs =
280       new HashMap<TableName, Set<String>>();
281 
282   private Map<TableName, Set<String>> skippedRegions = new HashMap<TableName, Set<String>>();
283 
284   /**
285    * List of orphaned table ZNodes
286    */
287   private Set<TableName> orphanedTableZNodes = new HashSet<TableName>();
288 
289   /**
290    * Constructor
291    *
292    * @param conf Configuration object
293    * @throws MasterNotRunningException if the master is not running
294    * @throws ZooKeeperConnectionException if unable to connect to ZooKeeper
295    */
296   public HBaseFsck(Configuration conf) throws MasterNotRunningException,
297       ZooKeeperConnectionException, IOException, ClassNotFoundException {
298     super(conf);
299     // make a copy, just to be sure we're not overriding someone else's config
300     setConf(HBaseConfiguration.create(getConf()));
301     // disable blockcache for tool invocation, see HBASE-10500
302     getConf().setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);
303     errors = getErrorReporter(conf);
304 
305     int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
306     executor = new ScheduledThreadPoolExecutor(numThreads, Threads.newDaemonThreadFactory("hbasefsck"));
307   }
308 
309   /**
310    * Constructor
311    *
312    * @param conf
313    *          Configuration object
314    * @throws MasterNotRunningException
315    *           if the master is not running
316    * @throws ZooKeeperConnectionException
317    *           if unable to connect to ZooKeeper
318    */
319   public HBaseFsck(Configuration conf, ExecutorService exec) throws MasterNotRunningException,
320       ZooKeeperConnectionException, IOException, ClassNotFoundException {
321     super(conf);
322     errors = getErrorReporter(getConf());
323     this.executor = exec;
324   }
325   
326   /**
327    * This method maintains a lock using a file. If the creation fails we return null
328    *
329    * @return FSDataOutputStream object corresponding to the newly opened lock file
330    * @throws IOException
331    */
332   private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
333     long start = EnvironmentEdgeManager.currentTimeMillis();
334     try {
335       FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
336       FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
337           HConstants.DATA_FILE_UMASK_KEY);
338       Path tmpDir = new Path(FSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY);
339       fs.mkdirs(tmpDir);
340       HBCK_LOCK_PATH = new Path(tmpDir, HBCK_LOCK_FILE);
341       final FSDataOutputStream out = FSUtils.create(fs, HBCK_LOCK_PATH, defaultPerms, false);
342       out.writeBytes(InetAddress.getLocalHost().toString());
343       out.flush();
344       return out;
345     } catch(RemoteException e) {
346       if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
347         return null;
348       } else {
349         throw e;
350       }
351     } finally {
352       long duration = EnvironmentEdgeManager.currentTimeMillis() - start;
353       if (duration > 30000) {
354         LOG.warn("Took " + duration + " milliseconds to obtain lock");
355         // took too long to obtain lock
356         return null;
357       }
358     }
359   }
360 
361   private void unlockHbck() {
362     if(hbckLockCleanup.compareAndSet(true, false)){
363       IOUtils.closeStream(hbckOutFd);
364       try{
365         FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
366       } catch(IOException ioe) {
367         LOG.warn("Failed to delete " + HBCK_LOCK_PATH);
368         LOG.debug(ioe);
369       }
370     }
371   }
372 
373   /**
374    * To repair region consistency, one must call connect() in order to repair
375    * online state.
376    */
377   public void connect() throws IOException {
378 
379     // Check if another instance of balancer is running
380     hbckOutFd = checkAndMarkRunningHbck();
381     if (hbckOutFd == null) {
382       setRetCode(-1);
383       LOG.error("Another instance of hbck is running, exiting this instance.[If you are sure" +
384 		      " no other instance is running, delete the lock file " +
385 		      HBCK_LOCK_PATH + " and rerun the tool]");
386       throw new IOException("Duplicate hbck - Abort");
387     }
388 
389     // Make sure to cleanup the lock
390     hbckLockCleanup.set(true);
391 
392     // Add a shutdown hook to this thread, incase user tries to
393     // kill the hbck with a ctrl-c, we want to cleanup the lock so that
394     // it is available for further calls
395     Runtime.getRuntime().addShutdownHook(new Thread() {
396       @Override
397       public void run() {
398           unlockHbck();
399       }
400     });
401     LOG.debug("Launching hbck");
402 
403     connection = HConnectionManager.createConnection(getConf());
404     admin = new HBaseAdmin(connection);
405     meta = new HTable(TableName.META_TABLE_NAME, connection);
406     status = admin.getClusterStatus();
407   }
408 
409   /**
410    * Get deployed regions according to the region servers.
411    */
412   private void loadDeployedRegions() throws IOException, InterruptedException {
413     // From the master, get a list of all known live region servers
414     Collection<ServerName> regionServers = status.getServers();
415     errors.print("Number of live region servers: " + regionServers.size());
416     if (details) {
417       for (ServerName rsinfo: regionServers) {
418         errors.print("  " + rsinfo.getServerName());
419       }
420     }
421 
422     // From the master, get a list of all dead region servers
423     Collection<ServerName> deadRegionServers = status.getDeadServerNames();
424     errors.print("Number of dead region servers: " + deadRegionServers.size());
425     if (details) {
426       for (ServerName name: deadRegionServers) {
427         errors.print("  " + name);
428       }
429     }
430 
431     // Print the current master name and state
432     errors.print("Master: " + status.getMaster());
433 
434     // Print the list of all backup masters
435     Collection<ServerName> backupMasters = status.getBackupMasters();
436     errors.print("Number of backup masters: " + backupMasters.size());
437     if (details) {
438       for (ServerName name: backupMasters) {
439         errors.print("  " + name);
440       }
441     }
442 
443     errors.print("Average load: " + status.getAverageLoad());
444     errors.print("Number of requests: " + status.getRequestsCount());
445     errors.print("Number of regions: " + status.getRegionsCount());
446 
447     Map<String, RegionState> rits = status.getRegionsInTransition();
448     errors.print("Number of regions in transition: " + rits.size());
449     if (details) {
450       for (RegionState state: rits.values()) {
451         errors.print("  " + state.toDescriptiveString());
452       }
453     }
454 
455     // Determine what's deployed
456     processRegionServers(regionServers);
457   }
458 
459   /**
460    * Clear the current state of hbck.
461    */
462   private void clearState() {
463     // Make sure regionInfo is empty before starting
464     fixes = 0;
465     regionInfoMap.clear();
466     emptyRegionInfoQualifiers.clear();
467     disabledTables.clear();
468     errors.clear();
469     tablesInfo.clear();
470     orphanHdfsDirs.clear();
471     skippedRegions.clear();
472   }
473 
474   /**
475    * This repair method analyzes hbase data in hdfs and repairs it to satisfy
476    * the table integrity rules.  HBase doesn't need to be online for this
477    * operation to work.
478    */
479   public void offlineHdfsIntegrityRepair() throws IOException, InterruptedException {
480     // Initial pass to fix orphans.
481     if (shouldCheckHdfs() && (shouldFixHdfsOrphans() || shouldFixHdfsHoles()
482         || shouldFixHdfsOverlaps() || shouldFixTableOrphans())) {
483       LOG.info("Loading regioninfos HDFS");
484       // if nothing is happening this should always complete in two iterations.
485       int maxIterations = getConf().getInt("hbase.hbck.integrityrepair.iterations.max", 3);
486       int curIter = 0;
487       do {
488         clearState(); // clears hbck state and reset fixes to 0 and.
489         // repair what's on HDFS
490         restoreHdfsIntegrity();
491         curIter++;// limit the number of iterations.
492       } while (fixes > 0 && curIter <= maxIterations);
493 
494       // Repairs should be done in the first iteration and verification in the second.
495       // If there are more than 2 passes, something funny has happened.
496       if (curIter > 2) {
497         if (curIter == maxIterations) {
498           LOG.warn("Exiting integrity repairs after max " + curIter + " iterations. "
499               + "Tables integrity may not be fully repaired!");
500         } else {
501           LOG.info("Successfully exiting integrity repairs after " + curIter + " iterations");
502         }
503       }
504     }
505   }
506 
507   /**
508    * This repair method requires the cluster to be online since it contacts
509    * region servers and the masters.  It makes each region's state in HDFS, in
510    * hbase:meta, and deployments consistent.
511    *
512    * @return If > 0 , number of errors detected, if < 0 there was an unrecoverable
513    * error.  If 0, we have a clean hbase.
514    */
515   public int onlineConsistencyRepair() throws IOException, KeeperException,
516     InterruptedException {
517     clearState();
518 
519     // get regions according to what is online on each RegionServer
520     loadDeployedRegions();
521     // check whether hbase:meta is deployed and online
522     recordMetaRegion();
523     // Check if hbase:meta is found only once and in the right place
524     if (!checkMetaRegion()) {
525       String errorMsg = "hbase:meta table is not consistent. ";
526       if (shouldFixAssignments()) {
527         errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state.";
528       } else {
529         errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency.";
530       }
531       errors.reportError(errorMsg + " Exiting...");
532       return -2;
533     }
534     // Not going with further consistency check for tables when hbase:meta itself is not consistent.
535     LOG.info("Loading regionsinfo from the hbase:meta table");
536     boolean success = loadMetaEntries();
537     if (!success) return -1;
538 
539     // Empty cells in hbase:meta?
540     reportEmptyMetaCells();
541 
542     // Check if we have to cleanup empty REGIONINFO_QUALIFIER rows from hbase:meta
543     if (shouldFixEmptyMetaCells()) {
544       fixEmptyMetaCells();
545     }
546 
547     // get a list of all tables that have not changed recently.
548     if (!checkMetaOnly) {
549       reportTablesInFlux();
550     }
551 
552     // load regiondirs and regioninfos from HDFS
553     if (shouldCheckHdfs()) {
554       LOG.info("Loading region directories from HDFS");
555       loadHdfsRegionDirs();
556       LOG.info("Loading region information from HDFS");
557       loadHdfsRegionInfos();
558     }
559 
560     // Get disabled tables from ZooKeeper
561     loadDisabledTables();
562 
563     // fix the orphan tables
564     fixOrphanTables();
565 
566     LOG.info("Checking and fixing region consistency");
567 
568     // Check and fix consistency
569     checkAndFixConsistency();
570 
571     // Check integrity (does not fix)
572     checkIntegrity();
573     return errors.getErrorList().size();
574   }
575 
576   /**
577    * Contacts the master and prints out cluster-wide information
578    * @return 0 on success, non-zero on failure
579    */
580   public int onlineHbck() throws IOException, KeeperException, InterruptedException, ServiceException {
581     // print hbase server version
582     errors.print("Version: " + status.getHBaseVersion());
583     offlineHdfsIntegrityRepair();
584 
585     // turn the balancer off
586     boolean oldBalancer = admin.setBalancerRunning(false, true);
587     try {
588       onlineConsistencyRepair();
589     }
590     finally {
591       admin.setBalancerRunning(oldBalancer, false);
592     }
593 
594     if (checkRegionBoundaries) {
595       checkRegionBoundaries();
596     }
597 
598     offlineReferenceFileRepair();
599 
600     checkAndFixTableLocks();
601 
602     // Check (and fix if requested) orphaned table ZNodes
603     checkAndFixOrphanedTableZNodes();
604 
605     // Remove the hbck lock
606     unlockHbck();
607 
608     // Print table summary
609     printTableSummary(tablesInfo);
610     return errors.summarize();
611   }
612 
613   public static byte[] keyOnly (byte[] b) {
614     if (b == null)
615       return b;
616     int rowlength = Bytes.toShort(b, 0);
617     byte[] result = new byte[rowlength];
618     System.arraycopy(b, Bytes.SIZEOF_SHORT, result, 0, rowlength);
619     return result;
620   }
621 
622   private static class RegionBoundariesInformation {
623     public byte [] regionName;
624     public byte [] metaFirstKey;
625     public byte [] metaLastKey;
626     public byte [] storesFirstKey;
627     public byte [] storesLastKey;
628     @Override
629     public String toString () {
630       return "regionName=" + Bytes.toStringBinary(regionName) +
631              "\nmetaFirstKey=" + Bytes.toStringBinary(metaFirstKey) +
632              "\nmetaLastKey=" + Bytes.toStringBinary(metaLastKey) +
633              "\nstoresFirstKey=" + Bytes.toStringBinary(storesFirstKey) +
634              "\nstoresLastKey=" + Bytes.toStringBinary(storesLastKey);
635     }
636   }
637 
638   public void checkRegionBoundaries() {
639     try {
640       ByteArrayComparator comparator = new ByteArrayComparator();
641       List<HRegionInfo> regions = MetaScanner.listAllRegions(getConf(), false);
642       final RegionBoundariesInformation currentRegionBoundariesInformation =
643           new RegionBoundariesInformation();
644       Path hbaseRoot = FSUtils.getRootDir(getConf());
645       for (HRegionInfo regionInfo : regions) {
646         Path tableDir = FSUtils.getTableDir(hbaseRoot, regionInfo.getTable());
647         currentRegionBoundariesInformation.regionName = regionInfo.getRegionName();
648         // For each region, get the start and stop key from the META and compare them to the
649         // same information from the Stores.
650         Path path = new Path(tableDir, regionInfo.getEncodedName());
651         FileSystem fs = path.getFileSystem(getConf());
652         FileStatus[] files = fs.listStatus(path);
653         // For all the column families in this region...
654         byte[] storeFirstKey = null;
655         byte[] storeLastKey = null;
656         for (FileStatus file : files) {
657           String fileName = file.getPath().toString();
658           fileName = fileName.substring(fileName.lastIndexOf("/") + 1);
659           if (!fileName.startsWith(".") && !fileName.endsWith("recovered.edits")) {
660             FileStatus[] storeFiles = fs.listStatus(file.getPath());
661             // For all the stores in this column family.
662             for (FileStatus storeFile : storeFiles) {
663               HFile.Reader reader = HFile.createReader(fs, storeFile.getPath(), new CacheConfig(
664                   getConf()), getConf());
665               if ((reader.getFirstKey() != null)
666                   && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
667                       reader.getFirstKey()) > 0))) {
668                 storeFirstKey = reader.getFirstKey();
669               }
670               if ((reader.getLastKey() != null)
671                   && ((storeLastKey == null) || (comparator.compare(storeLastKey,
672                       reader.getLastKey())) < 0)) {
673                 storeLastKey = reader.getLastKey();
674               }
675               reader.close();
676             }
677           }
678         }
679         currentRegionBoundariesInformation.metaFirstKey = regionInfo.getStartKey();
680         currentRegionBoundariesInformation.metaLastKey = regionInfo.getEndKey();
681         currentRegionBoundariesInformation.storesFirstKey = keyOnly(storeFirstKey);
682         currentRegionBoundariesInformation.storesLastKey = keyOnly(storeLastKey);
683         if (currentRegionBoundariesInformation.metaFirstKey.length == 0)
684           currentRegionBoundariesInformation.metaFirstKey = null;
685         if (currentRegionBoundariesInformation.metaLastKey.length == 0)
686           currentRegionBoundariesInformation.metaLastKey = null;
687 
688         // For a region to be correct, we need the META start key to be smaller or equal to the
689         // smallest start key from all the stores, and the start key from the next META entry to
690         // be bigger than the last key from all the current stores. First region start key is null;
691         // Last region end key is null; some regions can be empty and not have any store.
692 
693         boolean valid = true;
694         // Checking start key.
695         if ((currentRegionBoundariesInformation.storesFirstKey != null)
696             && (currentRegionBoundariesInformation.metaFirstKey != null)) {
697           valid = valid
698               && comparator.compare(currentRegionBoundariesInformation.storesFirstKey,
699                 currentRegionBoundariesInformation.metaFirstKey) >= 0;
700         }
701         // Checking stop key.
702         if ((currentRegionBoundariesInformation.storesLastKey != null)
703             && (currentRegionBoundariesInformation.metaLastKey != null)) {
704           valid = valid
705               && comparator.compare(currentRegionBoundariesInformation.storesLastKey,
706                 currentRegionBoundariesInformation.metaLastKey) < 0;
707         }
708         if (!valid) {
709           errors.reportError(ERROR_CODE.BOUNDARIES_ERROR, "Found issues with regions boundaries",
710             tablesInfo.get(regionInfo.getTable()));
711           LOG.warn("Region's boundaries not alligned between stores and META for:");
712           LOG.warn(currentRegionBoundariesInformation);
713         }
714       }
715     } catch (IOException e) {
716       LOG.error(e);
717     }
718   }
719 
720   /**
721    * Iterates through the list of all orphan/invalid regiondirs.
722    */
723   private void adoptHdfsOrphans(Collection<HbckInfo> orphanHdfsDirs) throws IOException {
724     for (HbckInfo hi : orphanHdfsDirs) {
725       LOG.info("Attempting to handle orphan hdfs dir: " + hi.getHdfsRegionDir());
726       adoptHdfsOrphan(hi);
727     }
728   }
729 
730   /**
731    * Orphaned regions are regions without a .regioninfo file in them.  We "adopt"
732    * these orphans by creating a new region, and moving the column families,
733    * recovered edits, HLogs, into the new region dir.  We determine the region
734    * startkey and endkeys by looking at all of the hfiles inside the column
735    * families to identify the min and max keys. The resulting region will
736    * likely violate table integrity but will be dealt with by merging
737    * overlapping regions.
738    */
739   @SuppressWarnings("deprecation")
740   private void adoptHdfsOrphan(HbckInfo hi) throws IOException {
741     Path p = hi.getHdfsRegionDir();
742     FileSystem fs = p.getFileSystem(getConf());
743     FileStatus[] dirs = fs.listStatus(p);
744     if (dirs == null) {
745       LOG.warn("Attempt to adopt ophan hdfs region skipped becuase no files present in " +
746           p + ". This dir could probably be deleted.");
747       return ;
748     }
749 
750     TableName tableName = hi.getTableName();
751     TableInfo tableInfo = tablesInfo.get(tableName);
752     Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not present!");
753     HTableDescriptor template = tableInfo.getHTD();
754 
755     // find min and max key values
756     Pair<byte[],byte[]> orphanRegionRange = null;
757     for (FileStatus cf : dirs) {
758       String cfName= cf.getPath().getName();
759       // TODO Figure out what the special dirs are
760       if (cfName.startsWith(".") || cfName.equals(HConstants.SPLIT_LOGDIR_NAME)) continue;
761 
762       FileStatus[] hfiles = fs.listStatus(cf.getPath());
763       for (FileStatus hfile : hfiles) {
764         byte[] start, end;
765         HFile.Reader hf = null;
766         try {
767           CacheConfig cacheConf = new CacheConfig(getConf());
768           hf = HFile.createReader(fs, hfile.getPath(), cacheConf, getConf());
769           hf.loadFileInfo();
770           KeyValue startKv = KeyValue.createKeyValueFromKey(hf.getFirstKey());
771           start = startKv.getRow();
772           KeyValue endKv = KeyValue.createKeyValueFromKey(hf.getLastKey());
773           end = endKv.getRow();
774         } catch (IOException ioe) {
775           LOG.warn("Problem reading orphan file " + hfile + ", skipping");
776           continue;
777         } catch (NullPointerException ioe) {
778           LOG.warn("Orphan file " + hfile + " is possibly corrupted HFile, skipping");
779           continue;
780         } finally {
781           if (hf != null) {
782             hf.close();
783           }
784         }
785 
786         // expand the range to include the range of all hfiles
787         if (orphanRegionRange == null) {
788           // first range
789           orphanRegionRange = new Pair<byte[], byte[]>(start, end);
790         } else {
791           // TODO add test
792 
793           // expand range only if the hfile is wider.
794           if (Bytes.compareTo(orphanRegionRange.getFirst(), start) > 0) {
795             orphanRegionRange.setFirst(start);
796           }
797           if (Bytes.compareTo(orphanRegionRange.getSecond(), end) < 0 ) {
798             orphanRegionRange.setSecond(end);
799           }
800         }
801       }
802     }
803     if (orphanRegionRange == null) {
804       LOG.warn("No data in dir " + p + ", sidelining data");
805       fixes++;
806       sidelineRegionDir(fs, hi);
807       return;
808     }
809     LOG.info("Min max keys are : [" + Bytes.toString(orphanRegionRange.getFirst()) + ", " +
810         Bytes.toString(orphanRegionRange.getSecond()) + ")");
811 
812     // create new region on hdfs.  move data into place.
813     HRegionInfo hri = new HRegionInfo(template.getTableName(), orphanRegionRange.getFirst(), 
814       Bytes.add(orphanRegionRange.getSecond(), new byte[1]));
815     LOG.info("Creating new region : " + hri);
816     HRegion region = HBaseFsckRepair.createHDFSRegionDir(getConf(), hri, template);
817     Path target = region.getRegionFileSystem().getRegionDir();
818 
819     // rename all the data to new region
820     mergeRegionDirs(target, hi);
821     fixes++;
822   }
823 
824   /**
825    * This method determines if there are table integrity errors in HDFS.  If
826    * there are errors and the appropriate "fix" options are enabled, the method
827    * will first correct orphan regions making them into legit regiondirs, and
828    * then reload to merge potentially overlapping regions.
829    *
830    * @return number of table integrity errors found
831    */
832   private int restoreHdfsIntegrity() throws IOException, InterruptedException {
833     // Determine what's on HDFS
834     LOG.info("Loading HBase regioninfo from HDFS...");
835     loadHdfsRegionDirs(); // populating regioninfo table.
836 
837     int errs = errors.getErrorList().size();
838     // First time just get suggestions.
839     tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
840     checkHdfsIntegrity(false, false);
841 
842     if (errors.getErrorList().size() == errs) {
843       LOG.info("No integrity errors.  We are done with this phase. Glorious.");
844       return 0;
845     }
846 
847     if (shouldFixHdfsOrphans() && orphanHdfsDirs.size() > 0) {
848       adoptHdfsOrphans(orphanHdfsDirs);
849       // TODO optimize by incrementally adding instead of reloading.
850     }
851 
852     // Make sure there are no holes now.
853     if (shouldFixHdfsHoles()) {
854       clearState(); // this also resets # fixes.
855       loadHdfsRegionDirs();
856       tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
857       tablesInfo = checkHdfsIntegrity(shouldFixHdfsHoles(), false);
858     }
859 
860     // Now we fix overlaps
861     if (shouldFixHdfsOverlaps()) {
862       // second pass we fix overlaps.
863       clearState(); // this also resets # fixes.
864       loadHdfsRegionDirs();
865       tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
866       tablesInfo = checkHdfsIntegrity(false, shouldFixHdfsOverlaps());
867     }
868 
869     return errors.getErrorList().size();
870   }
871 
872   /**
873    * Scan all the store file names to find any lingering reference files,
874    * which refer to some none-exiting files. If "fix" option is enabled,
875    * any lingering reference file will be sidelined if found.
876    * <p>
877    * Lingering reference file prevents a region from opening. It has to
878    * be fixed before a cluster can start properly.
879    */
880   private void offlineReferenceFileRepair() throws IOException {
881     Configuration conf = getConf();
882     Path hbaseRoot = FSUtils.getRootDir(conf);
883     FileSystem fs = hbaseRoot.getFileSystem(conf);
884     LOG.info("Computing mapping of all store files");
885     Map<String, Path> allFiles = FSUtils.getTableStoreFilePathMap(fs, hbaseRoot, errors);
886     errors.print("");
887     LOG.info("Validating mapping using HDFS state");
888     for (Path path: allFiles.values()) {
889       boolean isReference = false;
890       try {
891         isReference = StoreFileInfo.isReference(path);
892       } catch (Throwable t) {
893         // Ignore. Some files may not be store files at all.
894         // For example, files under .oldlogs folder in hbase:meta
895         // Warning message is already logged by
896         // StoreFile#isReference.
897       }
898       if (!isReference) continue;
899 
900       Path referredToFile = StoreFileInfo.getReferredToFile(path);
901       if (fs.exists(referredToFile)) continue;  // good, expected
902 
903       // Found a lingering reference file
904       errors.reportError(ERROR_CODE.LINGERING_REFERENCE_HFILE,
905         "Found lingering reference file " + path);
906       if (!shouldFixReferenceFiles()) continue;
907 
908       // Now, trying to fix it since requested
909       boolean success = false;
910       String pathStr = path.toString();
911 
912       // A reference file path should be like
913       // ${hbase.rootdir}/data/namespace/table_name/region_id/family_name/referred_file.region_name
914       // Up 5 directories to get the root folder.
915       // So the file will be sidelined to a similar folder structure.
916       int index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR);
917       for (int i = 0; index > 0 && i < 5; i++) {
918         index = pathStr.lastIndexOf(Path.SEPARATOR_CHAR, index - 1);
919       }
920       if (index > 0) {
921         Path rootDir = getSidelineDir();
922         Path dst = new Path(rootDir, pathStr.substring(index + 1));
923         fs.mkdirs(dst.getParent());
924         LOG.info("Trying to sildeline reference file "
925           + path + " to " + dst);
926         setShouldRerun();
927 
928         success = fs.rename(path, dst);
929       }
930       if (!success) {
931         LOG.error("Failed to sideline reference file " + path);
932       }
933     }
934   }
935 
936   /**
937    * TODO -- need to add tests for this.
938    */
939   private void reportEmptyMetaCells() {
940     errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: " +
941       emptyRegionInfoQualifiers.size());
942     if (details) {
943       for (Result r: emptyRegionInfoQualifiers) {
944         errors.print("  " + r);
945       }
946     }
947   }
948 
949   /**
950    * TODO -- need to add tests for this.
951    */
952   private void reportTablesInFlux() {
953     AtomicInteger numSkipped = new AtomicInteger(0);
954     HTableDescriptor[] allTables = getTables(numSkipped);
955     errors.print("Number of Tables: " + allTables.length);
956     if (details) {
957       if (numSkipped.get() > 0) {
958         errors.detail("Number of Tables in flux: " + numSkipped.get());
959       }
960       for (HTableDescriptor td : allTables) {
961         errors.detail("  Table: " + td.getTableName() + "\t" +
962                            (td.isReadOnly() ? "ro" : "rw") + "\t" +
963                             (td.isMetaRegion() ? "META" : "    ") + "\t" +
964                            " families: " + td.getFamilies().size());
965       }
966     }
967   }
968 
969   public ErrorReporter getErrors() {
970     return errors;
971   }
972 
973   /**
974    * Read the .regioninfo file from the file system.  If there is no
975    * .regioninfo, add it to the orphan hdfs region list.
976    */
977   private void loadHdfsRegioninfo(HbckInfo hbi) throws IOException {
978     Path regionDir = hbi.getHdfsRegionDir();
979     if (regionDir == null) {
980       LOG.warn("No HDFS region dir found: " + hbi + " meta=" + hbi.metaEntry);
981       return;
982     }
983 
984     if (hbi.hdfsEntry.hri != null) {
985       // already loaded data
986       return;
987     }
988 
989     FileSystem fs = FileSystem.get(getConf());
990     HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
991     LOG.debug("HRegionInfo read: " + hri.toString());
992     hbi.hdfsEntry.hri = hri;
993   }
994 
995   /**
996    * Exception thrown when a integrity repair operation fails in an
997    * unresolvable way.
998    */
999   public static class RegionRepairException extends IOException {
1000     private static final long serialVersionUID = 1L;
1001     final IOException ioe;
1002     public RegionRepairException(String s, IOException ioe) {
1003       super(s);
1004       this.ioe = ioe;
1005     }
1006   }
1007 
1008   /**
1009    * Populate hbi's from regionInfos loaded from file system.
1010    */
1011   private SortedMap<TableName, TableInfo> loadHdfsRegionInfos()
1012       throws IOException, InterruptedException {
1013     tablesInfo.clear(); // regenerating the data
1014     // generate region split structure
1015     Collection<HbckInfo> hbckInfos = regionInfoMap.values();
1016 
1017     // Parallelized read of .regioninfo files.
1018     List<WorkItemHdfsRegionInfo> hbis = new ArrayList<WorkItemHdfsRegionInfo>(hbckInfos.size());
1019     List<Future<Void>> hbiFutures;
1020 
1021     for (HbckInfo hbi : hbckInfos) {
1022       WorkItemHdfsRegionInfo work = new WorkItemHdfsRegionInfo(hbi, this, errors);
1023       hbis.add(work);
1024     }
1025 
1026     // Submit and wait for completion
1027     hbiFutures = executor.invokeAll(hbis);
1028 
1029     for(int i=0; i<hbiFutures.size(); i++) {
1030       WorkItemHdfsRegionInfo work = hbis.get(i);
1031       Future<Void> f = hbiFutures.get(i);
1032       try {
1033         f.get();
1034       } catch(ExecutionException e) {
1035         LOG.warn("Failed to read .regioninfo file for region " +
1036               work.hbi.getRegionNameAsString(), e.getCause());
1037       }
1038     }
1039 
1040     Path hbaseRoot = FSUtils.getRootDir(getConf());
1041     FileSystem fs = hbaseRoot.getFileSystem(getConf());
1042     // serialized table info gathering.
1043     for (HbckInfo hbi: hbckInfos) {
1044 
1045       if (hbi.getHdfsHRI() == null) {
1046         // was an orphan
1047         continue;
1048       }
1049 
1050 
1051       // get table name from hdfs, populate various HBaseFsck tables.
1052       TableName tableName = hbi.getTableName();
1053       if (tableName == null) {
1054         // There was an entry in hbase:meta not in the HDFS?
1055         LOG.warn("tableName was null for: " + hbi);
1056         continue;
1057       }
1058 
1059       TableInfo modTInfo = tablesInfo.get(tableName);
1060       if (modTInfo == null) {
1061         // only executed once per table.
1062         modTInfo = new TableInfo(tableName);
1063         tablesInfo.put(tableName, modTInfo);
1064         try {
1065           HTableDescriptor htd =
1066               FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
1067           modTInfo.htds.add(htd);
1068         } catch (IOException ioe) {
1069           if (!orphanTableDirs.containsKey(tableName)) {
1070             LOG.warn("Unable to read .tableinfo from " + hbaseRoot, ioe);
1071             //should only report once for each table
1072             errors.reportError(ERROR_CODE.NO_TABLEINFO_FILE,
1073                 "Unable to read .tableinfo from " + hbaseRoot + "/" + tableName);
1074             Set<String> columns = new HashSet<String>();
1075             orphanTableDirs.put(tableName, getColumnFamilyList(columns, hbi));
1076           }
1077         }
1078       }
1079       if (!hbi.isSkipChecks()) {
1080         modTInfo.addRegionInfo(hbi);
1081       }
1082     }
1083 
1084     loadTableInfosForTablesWithNoRegion();
1085     errors.print("");
1086 
1087     return tablesInfo;
1088   }
1089 
1090   /**
1091    * To get the column family list according to the column family dirs
1092    * @param columns
1093    * @param hbi
1094    * @return a set of column families
1095    * @throws IOException
1096    */
1097   private Set<String> getColumnFamilyList(Set<String> columns, HbckInfo hbi) throws IOException {
1098     Path regionDir = hbi.getHdfsRegionDir();
1099     FileSystem fs = regionDir.getFileSystem(getConf());
1100     FileStatus[] subDirs = fs.listStatus(regionDir, new FSUtils.FamilyDirFilter(fs));
1101     for (FileStatus subdir : subDirs) {
1102       String columnfamily = subdir.getPath().getName();
1103       columns.add(columnfamily);
1104     }
1105     return columns;
1106   }
1107 
1108   /**
1109    * To fabricate a .tableinfo file with following contents<br>
1110    * 1. the correct tablename <br>
1111    * 2. the correct colfamily list<br>
1112    * 3. the default properties for both {@link HTableDescriptor} and {@link HColumnDescriptor}<br>
1113    * @throws IOException
1114    */
1115   private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName,
1116       Set<String> columns) throws IOException {
1117     if (columns ==null || columns.isEmpty()) return false;
1118     HTableDescriptor htd = new HTableDescriptor(tableName);
1119     for (String columnfamimly : columns) {
1120       htd.addFamily(new HColumnDescriptor(columnfamimly));
1121     }
1122     fstd.createTableDescriptor(htd, true);
1123     return true;
1124   }
1125 
1126   /**
1127    * To fix the empty REGIONINFO_QUALIFIER rows from hbase:meta <br>
1128    * @throws IOException
1129    */
1130   public void fixEmptyMetaCells() throws IOException {
1131     if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) {
1132       LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows.");
1133       for (Result region : emptyRegionInfoQualifiers) {
1134         deleteMetaRegion(region.getRow());
1135         errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL);
1136       }
1137       emptyRegionInfoQualifiers.clear();
1138     }
1139   }
1140 
1141   /**
1142    * To fix orphan table by creating a .tableinfo file under tableDir <br>
1143    * 1. if TableInfo is cached, to recover the .tableinfo accordingly <br>
1144    * 2. else create a default .tableinfo file with following items<br>
1145    * &nbsp;2.1 the correct tablename <br>
1146    * &nbsp;2.2 the correct colfamily list<br>
1147    * &nbsp;2.3 the default properties for both {@link HTableDescriptor} and {@link HColumnDescriptor}<br>
1148    * @throws IOException
1149    */
1150   public void fixOrphanTables() throws IOException {
1151     if (shouldFixTableOrphans() && !orphanTableDirs.isEmpty()) {
1152 
1153       List<TableName> tmpList = new ArrayList<TableName>();
1154       tmpList.addAll(orphanTableDirs.keySet());
1155       HTableDescriptor[] htds = getHTableDescriptors(tmpList);
1156       Iterator<Entry<TableName, Set<String>>> iter =
1157           orphanTableDirs.entrySet().iterator();
1158       int j = 0;
1159       int numFailedCase = 0;
1160       FSTableDescriptors fstd = new FSTableDescriptors(getConf());
1161       while (iter.hasNext()) {
1162         Entry<TableName, Set<String>> entry =
1163             iter.next();
1164         TableName tableName = entry.getKey();
1165         LOG.info("Trying to fix orphan table error: " + tableName);
1166         if (j < htds.length) {
1167           if (tableName.equals(htds[j].getTableName())) {
1168             HTableDescriptor htd = htds[j];
1169             LOG.info("fixing orphan table: " + tableName + " from cache");
1170             fstd.createTableDescriptor(htd, true);
1171             j++;
1172             iter.remove();
1173           }
1174         } else {
1175           if (fabricateTableInfo(fstd, tableName, entry.getValue())) {
1176             LOG.warn("fixing orphan table: " + tableName + " with a default .tableinfo file");
1177             LOG.warn("Strongly recommend to modify the HTableDescriptor if necessary for: " + tableName);
1178             iter.remove();
1179           } else {
1180             LOG.error("Unable to create default .tableinfo for " + tableName + " while missing column family information");
1181             numFailedCase++;
1182           }
1183         }
1184         fixes++;
1185       }
1186 
1187       if (orphanTableDirs.isEmpty()) {
1188         // all orphanTableDirs are luckily recovered
1189         // re-run doFsck after recovering the .tableinfo file
1190         setShouldRerun();
1191         LOG.warn("Strongly recommend to re-run manually hfsck after all orphanTableDirs being fixed");
1192       } else if (numFailedCase > 0) {
1193         LOG.error("Failed to fix " + numFailedCase
1194             + " OrphanTables with default .tableinfo files");
1195       }
1196 
1197     }
1198     //cleanup the list
1199     orphanTableDirs.clear();
1200 
1201   }
1202 
1203   /**
1204    * This borrows code from MasterFileSystem.bootstrap()
1205    *
1206    * @return an open hbase:meta HRegion
1207    */
1208   private HRegion createNewMeta() throws IOException {
1209       Path rootdir = FSUtils.getRootDir(getConf());
1210     Configuration c = getConf();
1211     HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
1212     HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
1213     MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
1214     HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor);
1215     MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true);
1216     return meta;
1217   }
1218 
1219   /**
1220    * Generate set of puts to add to new meta.  This expects the tables to be
1221    * clean with no overlaps or holes.  If there are any problems it returns null.
1222    *
1223    * @return An array list of puts to do in bulk, null if tables have problems
1224    */
1225   private ArrayList<Put> generatePuts(
1226       SortedMap<TableName, TableInfo> tablesInfo) throws IOException {
1227     ArrayList<Put> puts = new ArrayList<Put>();
1228     boolean hasProblems = false;
1229     for (Entry<TableName, TableInfo> e : tablesInfo.entrySet()) {
1230       TableName name = e.getKey();
1231 
1232       // skip "hbase:meta"
1233       if (name.compareTo(TableName.META_TABLE_NAME) == 0) {
1234         continue;
1235       }
1236 
1237       TableInfo ti = e.getValue();
1238       for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap()
1239           .entrySet()) {
1240         Collection<HbckInfo> his = spl.getValue();
1241         int sz = his.size();
1242         if (sz != 1) {
1243           // problem
1244           LOG.error("Split starting at " + Bytes.toStringBinary(spl.getKey())
1245               + " had " +  sz + " regions instead of exactly 1." );
1246           hasProblems = true;
1247           continue;
1248         }
1249 
1250         // add the row directly to meta.
1251         HbckInfo hi = his.iterator().next();
1252         HRegionInfo hri = hi.getHdfsHRI(); // hi.metaEntry;
1253         Put p = MetaEditor.makePutFromRegionInfo(hri);
1254         puts.add(p);
1255       }
1256     }
1257     return hasProblems ? null : puts;
1258   }
1259 
1260   /**
1261    * Suggest fixes for each table
1262    */
1263   private void suggestFixes(
1264       SortedMap<TableName, TableInfo> tablesInfo) throws IOException {
1265     logParallelMerge();
1266     for (TableInfo tInfo : tablesInfo.values()) {
1267       TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
1268       tInfo.checkRegionChain(handler);
1269     }
1270   }
1271 
1272   /**
1273    * Rebuilds meta from information in hdfs/fs.  Depends on configuration
1274    * settings passed into hbck constructor to point to a particular fs/dir.
1275    *
1276    * @param fix flag that determines if method should attempt to fix holes
1277    * @return true if successful, false if attempt failed.
1278    */
1279   public boolean rebuildMeta(boolean fix) throws IOException,
1280       InterruptedException {
1281 
1282     // TODO check to make sure hbase is offline. (or at least the table
1283     // currently being worked on is off line)
1284 
1285     // Determine what's on HDFS
1286     LOG.info("Loading HBase regioninfo from HDFS...");
1287     loadHdfsRegionDirs(); // populating regioninfo table.
1288 
1289     int errs = errors.getErrorList().size();
1290     tablesInfo = loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
1291     checkHdfsIntegrity(false, false);
1292 
1293     // make sure ok.
1294     if (errors.getErrorList().size() != errs) {
1295       // While in error state, iterate until no more fixes possible
1296       while(true) {
1297         fixes = 0;
1298         suggestFixes(tablesInfo);
1299         errors.clear();
1300         loadHdfsRegionInfos(); // update tableInfos based on region info in fs.
1301         checkHdfsIntegrity(shouldFixHdfsHoles(), shouldFixHdfsOverlaps());
1302 
1303         int errCount = errors.getErrorList().size();
1304 
1305         if (fixes == 0) {
1306           if (errCount > 0) {
1307             return false; // failed to fix problems.
1308           } else {
1309             break; // no fixes and no problems? drop out and fix stuff!
1310           }
1311         }
1312       }
1313     }
1314 
1315     // we can rebuild, move old meta out of the way and start
1316     LOG.info("HDFS regioninfo's seems good.  Sidelining old hbase:meta");
1317     Path backupDir = sidelineOldMeta();
1318 
1319     LOG.info("Creating new hbase:meta");
1320     HRegion meta = createNewMeta();
1321 
1322     // populate meta
1323     List<Put> puts = generatePuts(tablesInfo);
1324     if (puts == null) {
1325       LOG.fatal("Problem encountered when creating new hbase:meta entries.  " +
1326         "You may need to restore the previously sidelined hbase:meta");
1327       return false;
1328     }
1329     meta.batchMutate(puts.toArray(new Put[puts.size()]));
1330     HRegion.closeHRegion(meta);
1331     LOG.info("Success! hbase:meta table rebuilt.");
1332     LOG.info("Old hbase:meta is moved into " + backupDir);
1333     return true;
1334   }
1335 
1336   /**
1337    * Log an appropriate message about whether or not overlapping merges are computed in parallel.
1338    */
1339   private void logParallelMerge() {
1340     if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) {
1341       LOG.info("Handling overlap merges in parallel. set hbasefsck.overlap.merge.parallel to" +
1342           " false to run serially.");
1343     } else {
1344       LOG.info("Handling overlap merges serially.  set hbasefsck.overlap.merge.parallel to" +
1345           " true to run in parallel.");
1346     }
1347   }
1348 
1349   private SortedMap<TableName, TableInfo> checkHdfsIntegrity(boolean fixHoles,
1350       boolean fixOverlaps) throws IOException {
1351     LOG.info("Checking HBase region split map from HDFS data...");
1352     logParallelMerge();
1353     for (TableInfo tInfo : tablesInfo.values()) {
1354       TableIntegrityErrorHandler handler;
1355       if (fixHoles || fixOverlaps) {
1356         handler = tInfo.new HDFSIntegrityFixer(tInfo, errors, getConf(),
1357           fixHoles, fixOverlaps);
1358       } else {
1359         handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
1360       }
1361       if (!tInfo.checkRegionChain(handler)) {
1362         // should dump info as well.
1363         errors.report("Found inconsistency in table " + tInfo.getName());
1364       }
1365     }
1366     return tablesInfo;
1367   }
1368 
1369   private Path getSidelineDir() throws IOException {
1370     if (sidelineDir == null) {
1371       Path hbaseDir = FSUtils.getRootDir(getConf());
1372       Path hbckDir = new Path(hbaseDir, HConstants.HBCK_SIDELINEDIR_NAME);
1373       sidelineDir = new Path(hbckDir, hbaseDir.getName() + "-"
1374           + startMillis);
1375     }
1376     return sidelineDir;
1377   }
1378 
1379   /**
1380    * Sideline a region dir (instead of deleting it)
1381    */
1382   Path sidelineRegionDir(FileSystem fs, HbckInfo hi) throws IOException {
1383     return sidelineRegionDir(fs, null, hi);
1384   }
1385 
1386   /**
1387    * Sideline a region dir (instead of deleting it)
1388    *
1389    * @param parentDir if specified, the region will be sidelined to
1390    * folder like .../parentDir/<table name>/<region name>. The purpose
1391    * is to group together similar regions sidelined, for example, those
1392    * regions should be bulk loaded back later on. If null, it is ignored.
1393    */
1394   Path sidelineRegionDir(FileSystem fs,
1395       String parentDir, HbckInfo hi) throws IOException {
1396     TableName tableName = hi.getTableName();
1397     Path regionDir = hi.getHdfsRegionDir();
1398 
1399     if (!fs.exists(regionDir)) {
1400       LOG.warn("No previous " + regionDir + " exists.  Continuing.");
1401       return null;
1402     }
1403 
1404     Path rootDir = getSidelineDir();
1405     if (parentDir != null) {
1406       rootDir = new Path(rootDir, parentDir);
1407     }
1408     Path sidelineTableDir= FSUtils.getTableDir(rootDir, tableName);
1409     Path sidelineRegionDir = new Path(sidelineTableDir, regionDir.getName());
1410     fs.mkdirs(sidelineRegionDir);
1411     boolean success = false;
1412     FileStatus[] cfs =  fs.listStatus(regionDir);
1413     if (cfs == null) {
1414       LOG.info("Region dir is empty: " + regionDir);
1415     } else {
1416       for (FileStatus cf : cfs) {
1417         Path src = cf.getPath();
1418         Path dst =  new Path(sidelineRegionDir, src.getName());
1419         if (fs.isFile(src)) {
1420           // simple file
1421           success = fs.rename(src, dst);
1422           if (!success) {
1423             String msg = "Unable to rename file " + src +  " to " + dst;
1424             LOG.error(msg);
1425             throw new IOException(msg);
1426           }
1427           continue;
1428         }
1429 
1430         // is a directory.
1431         fs.mkdirs(dst);
1432 
1433         LOG.info("Sidelining files from " + src + " into containing region " + dst);
1434         // FileSystem.rename is inconsistent with directories -- if the
1435         // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir,
1436         // it moves the src into the dst dir resulting in (foo/a/b).  If
1437         // the dst does not exist, and the src a dir, src becomes dst. (foo/b)
1438         FileStatus[] hfiles = fs.listStatus(src);
1439         if (hfiles != null && hfiles.length > 0) {
1440           for (FileStatus hfile : hfiles) {
1441             success = fs.rename(hfile.getPath(), dst);
1442             if (!success) {
1443               String msg = "Unable to rename file " + src +  " to " + dst;
1444               LOG.error(msg);
1445               throw new IOException(msg);
1446             }
1447           }
1448         }
1449         LOG.debug("Sideline directory contents:");
1450         debugLsr(sidelineRegionDir);
1451       }
1452     }
1453 
1454     LOG.info("Removing old region dir: " + regionDir);
1455     success = fs.delete(regionDir, true);
1456     if (!success) {
1457       String msg = "Unable to delete dir " + regionDir;
1458       LOG.error(msg);
1459       throw new IOException(msg);
1460     }
1461     return sidelineRegionDir;
1462   }
1463 
1464   /**
1465    * Side line an entire table.
1466    */
1467   void sidelineTable(FileSystem fs, TableName tableName, Path hbaseDir,
1468       Path backupHbaseDir) throws IOException {
1469     Path tableDir = FSUtils.getTableDir(hbaseDir, tableName);
1470     if (fs.exists(tableDir)) {
1471       Path backupTableDir= FSUtils.getTableDir(backupHbaseDir, tableName);
1472       fs.mkdirs(backupTableDir.getParent());
1473       boolean success = fs.rename(tableDir, backupTableDir);
1474       if (!success) {
1475         throw new IOException("Failed to move  " + tableName + " from "
1476             +  tableDir + " to " + backupTableDir);
1477       }
1478     } else {
1479       LOG.info("No previous " + tableName +  " exists.  Continuing.");
1480     }
1481   }
1482 
1483   /**
1484    * @return Path to backup of original directory
1485    */
1486   Path sidelineOldMeta() throws IOException {
1487     // put current hbase:meta aside.
1488     Path hbaseDir = FSUtils.getRootDir(getConf());
1489     FileSystem fs = hbaseDir.getFileSystem(getConf());
1490     Path backupDir = getSidelineDir();
1491     fs.mkdirs(backupDir);
1492 
1493     try {
1494       sidelineTable(fs, TableName.META_TABLE_NAME, hbaseDir, backupDir);
1495     } catch (IOException e) {
1496         LOG.fatal("... failed to sideline meta. Currently in inconsistent state.  To restore "
1497             + "try to rename hbase:meta in " + backupDir.getName() + " to "
1498             + hbaseDir.getName() + ".", e);
1499       throw e; // throw original exception
1500     }
1501     return backupDir;
1502   }
1503 
1504   /**
1505    * Load the list of disabled tables in ZK into local set.
1506    * @throws ZooKeeperConnectionException
1507    * @throws IOException
1508    */
1509   private void loadDisabledTables()
1510   throws ZooKeeperConnectionException, IOException {
1511     HConnectionManager.execute(new HConnectable<Void>(getConf()) {
1512       @Override
1513       public Void connect(HConnection connection) throws IOException {
1514         ZooKeeperWatcher zkw = createZooKeeperWatcher();
1515         try {
1516           for (TableName tableName :
1517               ZKTableReadOnly.getDisabledOrDisablingTables(zkw)) {
1518             disabledTables.add(tableName);
1519           }
1520         } catch (KeeperException ke) {
1521           throw new IOException(ke);
1522         } finally {
1523           zkw.close();
1524         }
1525         return null;
1526       }
1527     });
1528   }
1529 
1530   /**
1531    * Check if the specified region's table is disabled.
1532    */
1533   private boolean isTableDisabled(HRegionInfo regionInfo) {
1534     return disabledTables.contains(regionInfo.getTable());
1535   }
1536 
1537   /**
1538    * Scan HDFS for all regions, recording their information into
1539    * regionInfoMap
1540    */
1541   public void loadHdfsRegionDirs() throws IOException, InterruptedException {
1542     Path rootDir = FSUtils.getRootDir(getConf());
1543     FileSystem fs = rootDir.getFileSystem(getConf());
1544 
1545     // list all tables from HDFS
1546     List<FileStatus> tableDirs = Lists.newArrayList();
1547 
1548     boolean foundVersionFile = fs.exists(new Path(rootDir, HConstants.VERSION_FILE_NAME));
1549 
1550     List<Path> paths = FSUtils.getTableDirs(fs, rootDir);
1551     for (Path path : paths) {
1552       TableName tableName = FSUtils.getTableName(path);
1553        if ((!checkMetaOnly &&
1554            isTableIncluded(tableName)) ||
1555            tableName.equals(TableName.META_TABLE_NAME)) {
1556          tableDirs.add(fs.getFileStatus(path));
1557        }
1558     }
1559 
1560     // verify that version file exists
1561     if (!foundVersionFile) {
1562       errors.reportError(ERROR_CODE.NO_VERSION_FILE,
1563           "Version file does not exist in root dir " + rootDir);
1564       if (shouldFixVersionFile()) {
1565         LOG.info("Trying to create a new " + HConstants.VERSION_FILE_NAME
1566             + " file.");
1567         setShouldRerun();
1568         FSUtils.setVersion(fs, rootDir, getConf().getInt(
1569             HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000), getConf().getInt(
1570             HConstants.VERSION_FILE_WRITE_ATTEMPTS,
1571             HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
1572       }
1573     }
1574 
1575     // level 1:  <HBASE_DIR>/*
1576     List<WorkItemHdfsDir> dirs = new ArrayList<WorkItemHdfsDir>(tableDirs.size());
1577     List<Future<Void>> dirsFutures;
1578 
1579     for (FileStatus tableDir : tableDirs) {
1580       LOG.debug("Loading region dirs from " +tableDir.getPath());
1581       dirs.add(new WorkItemHdfsDir(this, fs, errors, tableDir));
1582     }
1583 
1584     // Invoke and wait for Callables to complete
1585     dirsFutures = executor.invokeAll(dirs);
1586 
1587     for(Future<Void> f: dirsFutures) {
1588       try {
1589         f.get();
1590       } catch(ExecutionException e) {
1591         LOG.warn("Could not load region dir " , e.getCause());
1592       }
1593     }
1594     errors.print("");
1595   }
1596 
1597   /**
1598    * Record the location of the hbase:meta region as found in ZooKeeper.
1599    */
1600   private boolean recordMetaRegion() throws IOException {
1601     HRegionLocation metaLocation = connection.locateRegion(
1602       TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW);
1603 
1604     // Check if Meta region is valid and existing
1605     if (metaLocation == null || metaLocation.getRegionInfo() == null ||
1606         metaLocation.getHostname() == null) {
1607       errors.reportError(ERROR_CODE.NULL_META_REGION,
1608         "META region or some of its attributes are null.");
1609       return false;
1610     }
1611     ServerName sn;
1612     try {
1613       sn = getMetaRegionServerName();
1614     } catch (KeeperException e) {
1615       throw new IOException(e);
1616     }
1617     MetaEntry m = new MetaEntry(metaLocation.getRegionInfo(), sn, System.currentTimeMillis());
1618     HbckInfo hbckInfo = regionInfoMap.get(metaLocation.getRegionInfo().getEncodedName());
1619     if (hbckInfo == null) {
1620       regionInfoMap.put(metaLocation.getRegionInfo().getEncodedName(), new HbckInfo(m));
1621     } else {
1622       hbckInfo.metaEntry = m;
1623     }
1624     return true;
1625   }
1626 
1627   private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
1628     return new ZooKeeperWatcher(getConf(), "hbase Fsck", new Abortable() {
1629       @Override
1630       public void abort(String why, Throwable e) {
1631         LOG.error(why, e);
1632         System.exit(1);
1633       }
1634 
1635       @Override
1636       public boolean isAborted() {
1637         return false;
1638       }
1639 
1640     });
1641   }
1642 
1643   private ServerName getMetaRegionServerName()
1644   throws IOException, KeeperException {
1645     ZooKeeperWatcher zkw = createZooKeeperWatcher();
1646     ServerName sn = null;
1647     try {
1648       sn = MetaRegionTracker.getMetaRegionLocation(zkw);
1649     } finally {
1650       zkw.close();
1651     }
1652     return sn;
1653   }
1654 
1655   /**
1656    * Contacts each regionserver and fetches metadata about regions.
1657    * @param regionServerList - the list of region servers to connect to
1658    * @throws IOException if a remote or network exception occurs
1659    */
1660   void processRegionServers(Collection<ServerName> regionServerList)
1661     throws IOException, InterruptedException {
1662 
1663     List<WorkItemRegion> workItems = new ArrayList<WorkItemRegion>(regionServerList.size());
1664     List<Future<Void>> workFutures;
1665 
1666     // loop to contact each region server in parallel
1667     for (ServerName rsinfo: regionServerList) {
1668       workItems.add(new WorkItemRegion(this, rsinfo, errors, connection));
1669     }
1670 
1671     workFutures = executor.invokeAll(workItems);
1672 
1673     for(int i=0; i<workFutures.size(); i++) {
1674       WorkItemRegion item = workItems.get(i);
1675       Future<Void> f = workFutures.get(i);
1676       try {
1677         f.get();
1678       } catch(ExecutionException e) {
1679         LOG.warn("Could not process regionserver " + item.rsinfo.getHostAndPort(),
1680             e.getCause());
1681       }
1682     }
1683   }
1684 
1685   /**
1686    * Check consistency of all regions that have been found in previous phases.
1687    */
1688   private void checkAndFixConsistency()
1689   throws IOException, KeeperException, InterruptedException {
1690     List<CheckRegionConsistencyWorkItem> workItems =
1691         new ArrayList<CheckRegionConsistencyWorkItem>(regionInfoMap.size());
1692     for (java.util.Map.Entry<String, HbckInfo> e: regionInfoMap.entrySet()) {
1693       workItems.add(new CheckRegionConsistencyWorkItem(e.getKey(), e.getValue()));
1694     }
1695     checkRegionConsistencyConcurrently(workItems);
1696 
1697     // If some regions is skipped during checkRegionConsistencyConcurrently() phase, we might
1698     // not get accurate state of the hbase if continuing. The config here allows users to tune
1699     // the tolerance of number of skipped region.
1700     // TODO: evaluate the consequence to continue the hbck operation without config.
1701     int terminateThreshold =  getConf().getInt("hbase.hbck.skipped.regions.limit", 0);
1702     int numOfSkippedRegions = skippedRegions.size();
1703     if (numOfSkippedRegions > 0 && numOfSkippedRegions > terminateThreshold) {
1704       throw new IOException(numOfSkippedRegions
1705         + " region(s) could not be checked or repaired.  See logs for detail.");
1706     }
1707   }
1708 
1709   /**
1710    * Check consistency of all regions using mulitple threads concurrently.
1711    */
1712   private void checkRegionConsistencyConcurrently(
1713     final List<CheckRegionConsistencyWorkItem> workItems)
1714     throws IOException, KeeperException, InterruptedException {
1715     if (workItems.isEmpty()) {
1716       return;  // nothing to check
1717     }
1718 
1719     List<Future<Void>> workFutures = executor.invokeAll(workItems);
1720     for(Future<Void> f: workFutures) {
1721       try {
1722         f.get();
1723       } catch(ExecutionException e1) {
1724         LOG.warn("Could not check region consistency " , e1.getCause());
1725         if (e1.getCause() instanceof IOException) {
1726           throw (IOException)e1.getCause();
1727         } else if (e1.getCause() instanceof KeeperException) {
1728           throw (KeeperException)e1.getCause();
1729         } else if (e1.getCause() instanceof InterruptedException) {
1730           throw (InterruptedException)e1.getCause();
1731         } else {
1732           throw new IOException(e1.getCause());
1733         }
1734       }
1735     }
1736   }
1737 
1738   class CheckRegionConsistencyWorkItem implements Callable<Void> {
1739     private final String key;
1740     private final HbckInfo hbi;
1741 
1742     CheckRegionConsistencyWorkItem(String key, HbckInfo hbi) {
1743       this.key = key;
1744       this.hbi = hbi;
1745     }
1746 
1747     @Override
1748     public synchronized Void call() throws Exception {
1749       try {
1750         checkRegionConsistency(key, hbi);
1751       } catch (Exception e) {
1752         // If the region is non-META region, skip this region and send warning/error message; if
1753         // the region is META region, we should not continue.
1754         LOG.warn("Unable to complete check or repair the region '" + hbi.getRegionNameAsString()
1755           + "'.", e);
1756         if (hbi.getHdfsHRI().isMetaRegion()) {
1757           throw e;
1758         }
1759         LOG.warn("Skip region '" + hbi.getRegionNameAsString() + "'");
1760         addSkippedRegion(hbi);
1761       }
1762       return null;
1763     }
1764   }
1765 
1766   private void addSkippedRegion(final HbckInfo hbi) {
1767     Set<String> skippedRegionNames = skippedRegions.get(hbi.getTableName());
1768     if (skippedRegionNames == null) {
1769       skippedRegionNames = new HashSet<String>();
1770     }
1771     skippedRegionNames.add(hbi.getRegionNameAsString());
1772     skippedRegions.put(hbi.getTableName(), skippedRegionNames);
1773   }
1774 
1775   private void preCheckPermission() throws IOException, AccessDeniedException {
1776     if (shouldIgnorePreCheckPermission()) {
1777       return;
1778     }
1779 
1780     Path hbaseDir = FSUtils.getRootDir(getConf());
1781     FileSystem fs = hbaseDir.getFileSystem(getConf());
1782     UserProvider userProvider = UserProvider.instantiate(getConf());
1783     UserGroupInformation ugi = userProvider.getCurrent().getUGI();
1784     FileStatus[] files = fs.listStatus(hbaseDir);
1785     for (FileStatus file : files) {
1786       try {
1787         FSUtils.checkAccess(ugi, file, FsAction.WRITE);
1788       } catch (AccessDeniedException ace) {
1789         LOG.warn("Got AccessDeniedException when preCheckPermission ", ace);
1790         errors.reportError(ERROR_CODE.WRONG_USAGE, "Current user " + ugi.getUserName()
1791           + " does not have write perms to " + file.getPath()
1792           + ". Please rerun hbck as hdfs user " + file.getOwner());
1793         throw ace;
1794       }
1795     }
1796   }
1797 
1798   /**
1799    * Deletes region from meta table
1800    */
1801   private void deleteMetaRegion(HbckInfo hi) throws IOException {
1802     deleteMetaRegion(hi.metaEntry.getRegionName());
1803   }
1804 
1805   /**
1806    * Deletes region from meta table
1807    */
1808   private void deleteMetaRegion(byte[] metaKey) throws IOException {
1809     Delete d = new Delete(metaKey);
1810     meta.delete(d);
1811     meta.flushCommits();
1812     LOG.info("Deleted " + Bytes.toString(metaKey) + " from META" );
1813   }
1814 
1815   /**
1816    * Reset the split parent region info in meta table
1817    */
1818   private void resetSplitParent(HbckInfo hi) throws IOException {
1819     RowMutations mutations = new RowMutations(hi.metaEntry.getRegionName());
1820     Delete d = new Delete(hi.metaEntry.getRegionName());
1821     d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER);
1822     d.deleteColumn(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER);
1823     mutations.add(d);
1824 
1825     HRegionInfo hri = new HRegionInfo(hi.metaEntry);
1826     hri.setOffline(false);
1827     hri.setSplit(false);
1828     Put p = MetaEditor.makePutFromRegionInfo(hri);
1829     mutations.add(p);
1830 
1831     meta.mutateRow(mutations);
1832     meta.flushCommits();
1833     LOG.info("Reset split parent " + hi.metaEntry.getRegionNameAsString() + " in META" );
1834   }
1835 
1836   /**
1837    * This backwards-compatibility wrapper for permanently offlining a region
1838    * that should not be alive.  If the region server does not support the
1839    * "offline" method, it will use the closest unassign method instead.  This
1840    * will basically work until one attempts to disable or delete the affected
1841    * table.  The problem has to do with in-memory only master state, so
1842    * restarting the HMaster or failing over to another should fix this.
1843    */
1844   private void offline(byte[] regionName) throws IOException {
1845     String regionString = Bytes.toStringBinary(regionName);
1846     if (!rsSupportsOffline) {
1847       LOG.warn("Using unassign region " + regionString
1848           + " instead of using offline method, you should"
1849           + " restart HMaster after these repairs");
1850       admin.unassign(regionName, true);
1851       return;
1852     }
1853 
1854     // first time we assume the rs's supports #offline.
1855     try {
1856       LOG.info("Offlining region " + regionString);
1857       admin.offline(regionName);
1858     } catch (IOException ioe) {
1859       String notFoundMsg = "java.lang.NoSuchMethodException: " +
1860         "org.apache.hadoop.hbase.master.HMaster.offline([B)";
1861       if (ioe.getMessage().contains(notFoundMsg)) {
1862         LOG.warn("Using unassign region " + regionString
1863             + " instead of using offline method, you should"
1864             + " restart HMaster after these repairs");
1865         rsSupportsOffline = false; // in the future just use unassign
1866         admin.unassign(regionName, true);
1867         return;
1868       }
1869       throw ioe;
1870     }
1871   }
1872 
1873   private void undeployRegions(HbckInfo hi) throws IOException, InterruptedException {
1874     for (OnlineEntry rse : hi.deployedEntries) {
1875       LOG.debug("Undeploy region "  + rse.hri + " from " + rse.hsa);
1876       try {
1877         HBaseFsckRepair.closeRegionSilentlyAndWait(admin, rse.hsa, rse.hri);
1878         offline(rse.hri.getRegionName());
1879       } catch (IOException ioe) {
1880         LOG.warn("Got exception when attempting to offline region "
1881             + Bytes.toString(rse.hri.getRegionName()), ioe);
1882       }
1883     }
1884   }
1885 
1886   /**
1887    * Attempts to undeploy a region from a region server based in information in
1888    * META.  Any operations that modify the file system should make sure that
1889    * its corresponding region is not deployed to prevent data races.
1890    *
1891    * A separate call is required to update the master in-memory region state
1892    * kept in the AssignementManager.  Because disable uses this state instead of
1893    * that found in META, we can't seem to cleanly disable/delete tables that
1894    * have been hbck fixed.  When used on a version of HBase that does not have
1895    * the offline ipc call exposed on the master (<0.90.5, <0.92.0) a master
1896    * restart or failover may be required.
1897    */
1898   private void closeRegion(HbckInfo hi) throws IOException, InterruptedException {
1899     if (hi.metaEntry == null && hi.hdfsEntry == null) {
1900       undeployRegions(hi);
1901       return;
1902     }
1903 
1904     // get assignment info and hregioninfo from meta.
1905     Get get = new Get(hi.getRegionName());
1906     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1907     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
1908     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER);
1909     Result r = meta.get(get);
1910     ServerName serverName = HRegionInfo.getServerName(r);
1911     if (serverName == null) {
1912       errors.reportError("Unable to close region "
1913           + hi.getRegionNameAsString() +  " because meta does not "
1914           + "have handle to reach it.");
1915       return;
1916     }
1917 
1918     HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
1919     if (hri == null) {
1920       LOG.warn("Unable to close region " + hi.getRegionNameAsString()
1921           + " because hbase:meta had invalid or missing "
1922           + HConstants.CATALOG_FAMILY_STR + ":"
1923           + Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
1924           + " qualifier value.");
1925       return;
1926     }
1927 
1928     // close the region -- close files and remove assignment
1929     HBaseFsckRepair.closeRegionSilentlyAndWait(admin, serverName, hri);
1930   }
1931 
1932   private void tryAssignmentRepair(HbckInfo hbi, String msg) throws IOException,
1933     KeeperException, InterruptedException {
1934     // If we are trying to fix the errors
1935     if (shouldFixAssignments()) {
1936       errors.print(msg);
1937       undeployRegions(hbi);
1938       setShouldRerun();
1939       HRegionInfo hri = hbi.getHdfsHRI();
1940       if (hri == null) {
1941         hri = hbi.metaEntry;
1942       }
1943       HBaseFsckRepair.fixUnassigned(admin, hri);
1944       HBaseFsckRepair.waitUntilAssigned(admin, hri);
1945     }
1946   }
1947 
1948   /**
1949    * Check a single region for consistency and correct deployment.
1950    */
1951   private void checkRegionConsistency(final String key, final HbckInfo hbi)
1952   throws IOException, KeeperException, InterruptedException {
1953     String descriptiveName = hbi.toString();
1954 
1955     boolean inMeta = hbi.metaEntry != null;
1956     // In case not checking HDFS, assume the region is on HDFS
1957     boolean inHdfs = !shouldCheckHdfs() || hbi.getHdfsRegionDir() != null;
1958     boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null;
1959     boolean isDeployed = !hbi.deployedOn.isEmpty();
1960     boolean isMultiplyDeployed = hbi.deployedOn.size() > 1;
1961     boolean deploymentMatchesMeta =
1962       hasMetaAssignment && isDeployed && !isMultiplyDeployed &&
1963       hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0));
1964     boolean splitParent =
1965       (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
1966     boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
1967     boolean recentlyModified = inHdfs &&
1968       hbi.getModTime() + timelag > EnvironmentEdgeManager.currentTimeMillis();
1969 
1970     // ========== First the healthy cases =============
1971     if (hbi.containsOnlyHdfsEdits()) {
1972       return;
1973     }
1974     if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) {
1975       return;
1976     } else if (inMeta && inHdfs && !shouldBeDeployed && !isDeployed) {
1977       LOG.info("Region " + descriptiveName + " is in META, and in a disabled " +
1978         "tabled that is not deployed");
1979       return;
1980     } else if (recentlyModified) {
1981       LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
1982       return;
1983     }
1984     // ========== Cases where the region is not in hbase:meta =============
1985     else if (!inMeta && !inHdfs && !isDeployed) {
1986       // We shouldn't have record of this region at all then!
1987       assert false : "Entry for region with no data";
1988     } else if (!inMeta && !inHdfs && isDeployed) {
1989       errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region "
1990           + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " +
1991           "deployed on " + Joiner.on(", ").join(hbi.deployedOn));
1992       if (shouldFixAssignments()) {
1993         undeployRegions(hbi);
1994       }
1995 
1996     } else if (!inMeta && inHdfs && !isDeployed) {
1997       if (hbi.isMerged()) {
1998         // This region has already been merged, the remaining hdfs file will be
1999         // cleaned by CatalogJanitor later
2000         hbi.setSkipChecks(true);
2001         LOG.info("Region " + descriptiveName
2002             + " got merge recently, its file(s) will be cleaned by CatalogJanitor later");
2003         return;
2004       }
2005       errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region "
2006           + descriptiveName + " on HDFS, but not listed in hbase:meta " +
2007           "or deployed on any region server");
2008       // restore region consistency of an adopted orphan
2009       if (shouldFixMeta()) {
2010         if (!hbi.isHdfsRegioninfoPresent()) {
2011           LOG.error("Region " + hbi.getHdfsHRI() + " could have been repaired"
2012               +  " in table integrity repair phase if -fixHdfsOrphans was" +
2013               " used.");
2014           return;
2015         }
2016 
2017         HRegionInfo hri = hbi.getHdfsHRI();
2018         TableInfo tableInfo = tablesInfo.get(hri.getTable());
2019         for (HRegionInfo region : tableInfo.getRegionsFromMeta()) {
2020           if (Bytes.compareTo(region.getStartKey(), hri.getStartKey()) <= 0
2021               && (region.getEndKey().length == 0 || Bytes.compareTo(region.getEndKey(),
2022                 hri.getEndKey()) >= 0)
2023               && Bytes.compareTo(region.getStartKey(), hri.getEndKey()) <= 0) {
2024             if(region.isSplit() || region.isOffline()) continue;
2025             Path regionDir = hbi.getHdfsRegionDir();
2026             FileSystem fs = regionDir.getFileSystem(getConf());
2027             List<Path> familyDirs = FSUtils.getFamilyDirs(fs, regionDir);
2028             for (Path familyDir : familyDirs) {
2029               List<Path> referenceFilePaths = FSUtils.getReferenceFilePaths(fs, familyDir);
2030               for (Path referenceFilePath : referenceFilePaths) {
2031                 Path parentRegionDir =
2032                     StoreFileInfo.getReferredToFile(referenceFilePath).getParent().getParent();
2033                 if (parentRegionDir.toString().endsWith(region.getEncodedName())) {
2034                   LOG.warn(hri + " start and stop keys are in the range of " + region
2035                       + ". The region might not be cleaned up from hdfs when region " + region
2036                       + " split failed. Hence deleting from hdfs.");
2037                   HRegionFileSystem.deleteRegionFromFileSystem(getConf(), fs,
2038                     regionDir.getParent(), hri);
2039                   return;
2040                 }
2041               }
2042             }
2043           }
2044         }
2045 
2046         LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI());
2047         HBaseFsckRepair.fixMetaHoleOnline(getConf(), hbi.getHdfsHRI());
2048 
2049         tryAssignmentRepair(hbi, "Trying to reassign region...");
2050       }
2051 
2052     } else if (!inMeta && inHdfs && isDeployed) {
2053       errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
2054           + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2055       debugLsr(hbi.getHdfsRegionDir());
2056       if (shouldFixMeta()) {
2057         if (!hbi.isHdfsRegioninfoPresent()) {
2058           LOG.error("This should have been repaired in table integrity repair phase");
2059           return;
2060         }
2061 
2062         LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI());
2063         HBaseFsckRepair.fixMetaHoleOnline(getConf(), hbi.getHdfsHRI());
2064 
2065         tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2066       }
2067 
2068     // ========== Cases where the region is in hbase:meta =============
2069     } else if (inMeta && inHdfs && !isDeployed && splitParent) {
2070       // check whether this is an actual error, or just transient state where parent
2071       // is not cleaned
2072       if (hbi.metaEntry.splitA != null && hbi.metaEntry.splitB != null) {
2073         // check that split daughters are there
2074         HbckInfo infoA = this.regionInfoMap.get(hbi.metaEntry.splitA.getEncodedName());
2075         HbckInfo infoB = this.regionInfoMap.get(hbi.metaEntry.splitB.getEncodedName());
2076         if (infoA != null && infoB != null) {
2077           // we already processed or will process daughters. Move on, nothing to see here.
2078           hbi.setSkipChecks(true);
2079           return;
2080         }
2081       }
2082       errors.reportError(ERROR_CODE.LINGERING_SPLIT_PARENT, "Region "
2083           + descriptiveName + " is a split parent in META, in HDFS, "
2084           + "and not deployed on any region server. This could be transient.");
2085       if (shouldFixSplitParents()) {
2086         setShouldRerun();
2087         resetSplitParent(hbi);
2088       }
2089     } else if (inMeta && !inHdfs && !isDeployed) {
2090       errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region "
2091           + descriptiveName + " found in META, but not in HDFS "
2092           + "or deployed on any region server.");
2093       if (shouldFixMeta()) {
2094         deleteMetaRegion(hbi);
2095       }
2096     } else if (inMeta && !inHdfs && isDeployed) {
2097       errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName
2098           + " found in META, but not in HDFS, " +
2099           "and deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2100       // We treat HDFS as ground truth.  Any information in meta is transient
2101       // and equivalent data can be regenerated.  So, lets unassign and remove
2102       // these problems from META.
2103       if (shouldFixAssignments()) {
2104         errors.print("Trying to fix unassigned region...");
2105         undeployRegions(hbi);
2106       }
2107       if (shouldFixMeta()) {
2108         // wait for it to complete
2109         deleteMetaRegion(hbi);
2110       }
2111     } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) {
2112       errors.reportError(ERROR_CODE.NOT_DEPLOYED, "Region " + descriptiveName
2113           + " not deployed on any region server.");
2114       tryAssignmentRepair(hbi, "Trying to fix unassigned region...");
2115     } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) {
2116       errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED,
2117           "Region " + descriptiveName + " should not be deployed according " +
2118           "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn));
2119       if (shouldFixAssignments()) {
2120         errors.print("Trying to close the region " + descriptiveName);
2121         setShouldRerun();
2122         HBaseFsckRepair.fixMultiAssignment(admin, hbi.metaEntry, hbi.deployedOn);
2123       }
2124     } else if (inMeta && inHdfs && isMultiplyDeployed) {
2125       errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName
2126           + " is listed in hbase:meta on region server " + hbi.metaEntry.regionServer
2127           + " but is multiply assigned to region servers " +
2128           Joiner.on(", ").join(hbi.deployedOn));
2129       // If we are trying to fix the errors
2130       if (shouldFixAssignments()) {
2131         errors.print("Trying to fix assignment error...");
2132         setShouldRerun();
2133         HBaseFsckRepair.fixMultiAssignment(admin, hbi.metaEntry, hbi.deployedOn);
2134       }
2135     } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
2136       errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region "
2137           + descriptiveName + " listed in hbase:meta on region server " +
2138           hbi.metaEntry.regionServer + " but found on region server " +
2139           hbi.deployedOn.get(0));
2140       // If we are trying to fix the errors
2141       if (shouldFixAssignments()) {
2142         errors.print("Trying to fix assignment error...");
2143         setShouldRerun();
2144         HBaseFsckRepair.fixMultiAssignment(admin, hbi.metaEntry, hbi.deployedOn);
2145         HBaseFsckRepair.waitUntilAssigned(admin, hbi.getHdfsHRI());
2146       }
2147     } else {
2148       errors.reportError(ERROR_CODE.UNKNOWN, "Region " + descriptiveName +
2149           " is in an unforeseen state:" +
2150           " inMeta=" + inMeta +
2151           " inHdfs=" + inHdfs +
2152           " isDeployed=" + isDeployed +
2153           " isMultiplyDeployed=" + isMultiplyDeployed +
2154           " deploymentMatchesMeta=" + deploymentMatchesMeta +
2155           " shouldBeDeployed=" + shouldBeDeployed);
2156     }
2157   }
2158 
2159   /**
2160    * Checks tables integrity. Goes over all regions and scans the tables.
2161    * Collects all the pieces for each table and checks if there are missing,
2162    * repeated or overlapping ones.
2163    * @throws IOException
2164    */
2165   SortedMap<TableName, TableInfo> checkIntegrity() throws IOException {
2166     tablesInfo = new TreeMap<TableName,TableInfo> ();
2167     List<HbckInfo> noHDFSRegionInfos = new ArrayList<HbckInfo>();
2168     LOG.debug("There are " + regionInfoMap.size() + " region info entries");
2169     for (HbckInfo hbi : regionInfoMap.values()) {
2170       // Check only valid, working regions
2171       if (hbi.metaEntry == null) {
2172         // this assumes that consistency check has run loadMetaEntry
2173         noHDFSRegionInfos.add(hbi);
2174         Path p = hbi.getHdfsRegionDir();
2175         if (p == null) {
2176           errors.report("No regioninfo in Meta or HDFS. " + hbi);
2177         }
2178 
2179         // TODO test.
2180         continue;
2181       }
2182       if (hbi.metaEntry.regionServer == null) {
2183         errors.detail("Skipping region because no region server: " + hbi);
2184         continue;
2185       }
2186       if (hbi.metaEntry.isOffline()) {
2187         errors.detail("Skipping region because it is offline: " + hbi);
2188         continue;
2189       }
2190       if (hbi.containsOnlyHdfsEdits()) {
2191         errors.detail("Skipping region because it only contains edits" + hbi);
2192         continue;
2193       }
2194 
2195       // Missing regionDir or over-deployment is checked elsewhere. Include
2196       // these cases in modTInfo, so we can evaluate those regions as part of
2197       // the region chain in META
2198       //if (hbi.foundRegionDir == null) continue;
2199       //if (hbi.deployedOn.size() != 1) continue;
2200       if (hbi.deployedOn.size() == 0) continue;
2201 
2202       // We should be safe here
2203       TableName tableName = hbi.metaEntry.getTable();
2204       TableInfo modTInfo = tablesInfo.get(tableName);
2205       if (modTInfo == null) {
2206         modTInfo = new TableInfo(tableName);
2207       }
2208       for (ServerName server : hbi.deployedOn) {
2209         modTInfo.addServer(server);
2210       }
2211 
2212       if (!hbi.isSkipChecks()) {
2213         modTInfo.addRegionInfo(hbi);
2214       }
2215 
2216       tablesInfo.put(tableName, modTInfo);
2217     }
2218 
2219     loadTableInfosForTablesWithNoRegion();
2220 
2221     logParallelMerge();
2222     for (TableInfo tInfo : tablesInfo.values()) {
2223       TableIntegrityErrorHandler handler = tInfo.new IntegrityFixSuggester(tInfo, errors);
2224       if (!tInfo.checkRegionChain(handler)) {
2225         errors.report("Found inconsistency in table " + tInfo.getName());
2226       }
2227     }
2228     return tablesInfo;
2229   }
2230 
2231   /** Loads table info's for tables that may not have been included, since there are no
2232    * regions reported for the table, but table dir is there in hdfs
2233    */
2234   private void loadTableInfosForTablesWithNoRegion() throws IOException {
2235     Map<String, HTableDescriptor> allTables = new FSTableDescriptors(getConf()).getAll();
2236     for (HTableDescriptor htd : allTables.values()) {
2237       if (checkMetaOnly && !htd.isMetaTable()) {
2238         continue;
2239       }
2240 
2241       TableName tableName = htd.getTableName();
2242       if (isTableIncluded(tableName) && !tablesInfo.containsKey(tableName)) {
2243         TableInfo tableInfo = new TableInfo(tableName);
2244         tableInfo.htds.add(htd);
2245         tablesInfo.put(htd.getTableName(), tableInfo);
2246       }
2247     }
2248   }
2249 
2250   /**
2251    * Merge hdfs data by moving from contained HbckInfo into targetRegionDir.
2252    * @return number of file move fixes done to merge regions.
2253    */
2254   public int mergeRegionDirs(Path targetRegionDir, HbckInfo contained) throws IOException {
2255     int fileMoves = 0;
2256     String thread = Thread.currentThread().getName();
2257     LOG.debug("[" + thread + "] Contained region dir after close and pause");
2258     debugLsr(contained.getHdfsRegionDir());
2259 
2260     // rename the contained into the container.
2261     FileSystem fs = targetRegionDir.getFileSystem(getConf());
2262     FileStatus[] dirs = null;
2263     try {
2264       dirs = fs.listStatus(contained.getHdfsRegionDir());
2265     } catch (FileNotFoundException fnfe) {
2266       // region we are attempting to merge in is not present!  Since this is a merge, there is
2267       // no harm skipping this region if it does not exist.
2268       if (!fs.exists(contained.getHdfsRegionDir())) {
2269         LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2270             + " is missing. Assuming already sidelined or moved.");
2271       } else {
2272         sidelineRegionDir(fs, contained);
2273       }
2274       return fileMoves;
2275     }
2276 
2277     if (dirs == null) {
2278       if (!fs.exists(contained.getHdfsRegionDir())) {
2279         LOG.warn("[" + thread + "] HDFS region dir " + contained.getHdfsRegionDir()
2280             + " already sidelined.");
2281       } else {
2282         sidelineRegionDir(fs, contained);
2283       }
2284       return fileMoves;
2285     }
2286 
2287     for (FileStatus cf : dirs) {
2288       Path src = cf.getPath();
2289       Path dst =  new Path(targetRegionDir, src.getName());
2290 
2291       if (src.getName().equals(HRegionFileSystem.REGION_INFO_FILE)) {
2292         // do not copy the old .regioninfo file.
2293         continue;
2294       }
2295 
2296       if (src.getName().equals(HConstants.HREGION_OLDLOGDIR_NAME)) {
2297         // do not copy the .oldlogs files
2298         continue;
2299       }
2300 
2301       LOG.info("[" + thread + "] Moving files from " + src + " into containing region " + dst);
2302       // FileSystem.rename is inconsistent with directories -- if the
2303       // dst (foo/a) exists and is a dir, and the src (foo/b) is a dir,
2304       // it moves the src into the dst dir resulting in (foo/a/b).  If
2305       // the dst does not exist, and the src a dir, src becomes dst. (foo/b)
2306       for (FileStatus hfile : fs.listStatus(src)) {
2307         boolean success = fs.rename(hfile.getPath(), dst);
2308         if (success) {
2309           fileMoves++;
2310         }
2311       }
2312       LOG.debug("[" + thread + "] Sideline directory contents:");
2313       debugLsr(targetRegionDir);
2314     }
2315 
2316     // if all success.
2317     sidelineRegionDir(fs, contained);
2318     LOG.info("[" + thread + "] Sidelined region dir "+ contained.getHdfsRegionDir() + " into " +
2319         getSidelineDir());
2320     debugLsr(contained.getHdfsRegionDir());
2321 
2322     return fileMoves;
2323   }
2324 
2325 
2326   static class WorkItemOverlapMerge implements Callable<Void> {
2327     private TableIntegrityErrorHandler handler;
2328     Collection<HbckInfo> overlapgroup;
2329 
2330     WorkItemOverlapMerge(Collection<HbckInfo> overlapgroup, TableIntegrityErrorHandler handler) {
2331       this.handler = handler;
2332       this.overlapgroup = overlapgroup;
2333     }
2334 
2335     @Override
2336     public Void call() throws Exception {
2337       handler.handleOverlapGroup(overlapgroup);
2338       return null;
2339     }
2340   };
2341 
2342 
2343   /**
2344    * Maintain information about a particular table.
2345    */
2346   public class TableInfo {
2347     TableName tableName;
2348     TreeSet <ServerName> deployedOn;
2349 
2350     // backwards regions
2351     final List<HbckInfo> backwards = new ArrayList<HbckInfo>();
2352 
2353     // sidelined big overlapped regions
2354     final Map<Path, HbckInfo> sidelinedRegions = new HashMap<Path, HbckInfo>();
2355 
2356     // region split calculator
2357     final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<HbckInfo>(cmp);
2358 
2359     // Histogram of different HTableDescriptors found.  Ideally there is only one!
2360     final Set<HTableDescriptor> htds = new HashSet<HTableDescriptor>();
2361 
2362     // key = start split, values = set of splits in problem group
2363     final Multimap<byte[], HbckInfo> overlapGroups =
2364       TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp);
2365 
2366     // list of regions derived from meta entries.
2367     private ImmutableList<HRegionInfo> regionsFromMeta = null;
2368 
2369     TableInfo(TableName name) {
2370       this.tableName = name;
2371       deployedOn = new TreeSet <ServerName>();
2372     }
2373 
2374     /**
2375      * @return descriptor common to all regions.  null if are none or multiple!
2376      */
2377     private HTableDescriptor getHTD() {
2378       if (htds.size() == 1) {
2379         return (HTableDescriptor)htds.toArray()[0];
2380       } else {
2381         LOG.error("None/Multiple table descriptors found for table '"
2382           + tableName + "' regions: " + htds);
2383       }
2384       return null;
2385     }
2386 
2387     public void addRegionInfo(HbckInfo hir) {
2388       if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) {
2389         // end key is absolute end key, just add it.
2390         sc.add(hir);
2391         return;
2392       }
2393 
2394       // if not the absolute end key, check for cycle
2395       if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) {
2396         errors.reportError(
2397             ERROR_CODE.REGION_CYCLE,
2398             String.format("The endkey for this region comes before the "
2399                 + "startkey, startkey=%s, endkey=%s",
2400                 Bytes.toStringBinary(hir.getStartKey()),
2401                 Bytes.toStringBinary(hir.getEndKey())), this, hir);
2402         backwards.add(hir);
2403         return;
2404       }
2405 
2406       // main case, add to split calculator
2407       sc.add(hir);
2408     }
2409 
2410     public void addServer(ServerName server) {
2411       this.deployedOn.add(server);
2412     }
2413 
2414     public TableName getName() {
2415       return tableName;
2416     }
2417 
2418     public int getNumRegions() {
2419       return sc.getStarts().size() + backwards.size();
2420     }
2421 
2422     public synchronized ImmutableList<HRegionInfo> getRegionsFromMeta() {
2423       // lazy loaded, synchronized to ensure a single load
2424       if (regionsFromMeta == null) {
2425         List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2426         for (HbckInfo h : HBaseFsck.this.regionInfoMap.values()) {
2427           if (tableName.equals(h.getTableName())) {
2428             if (h.metaEntry != null) {
2429               regions.add((HRegionInfo) h.metaEntry);
2430             }
2431           }
2432         }
2433         regionsFromMeta = Ordering.natural().immutableSortedCopy(regions);
2434       }
2435       
2436       return regionsFromMeta;
2437     }
2438 
2439     private class IntegrityFixSuggester extends TableIntegrityErrorHandlerImpl {
2440       ErrorReporter errors;
2441 
2442       IntegrityFixSuggester(TableInfo ti, ErrorReporter errors) {
2443         this.errors = errors;
2444         setTableInfo(ti);
2445       }
2446 
2447       @Override
2448       public void handleRegionStartKeyNotEmpty(HbckInfo hi) throws IOException{
2449         errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
2450             "First region should start with an empty key.  You need to "
2451             + " create a new region and regioninfo in HDFS to plug the hole.",
2452             getTableInfo(), hi);
2453       }
2454 
2455       @Override
2456       public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException {
2457         errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
2458             "Last region should end with an empty key. You need to "
2459                 + "create a new region and regioninfo in HDFS to plug the hole.", getTableInfo());
2460       }
2461 
2462       @Override
2463       public void handleDegenerateRegion(HbckInfo hi) throws IOException{
2464         errors.reportError(ERROR_CODE.DEGENERATE_REGION,
2465             "Region has the same start and end key.", getTableInfo(), hi);
2466       }
2467 
2468       @Override
2469       public void handleDuplicateStartKeys(HbckInfo r1, HbckInfo r2) throws IOException{
2470         byte[] key = r1.getStartKey();
2471         // dup start key
2472         errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
2473             "Multiple regions have the same startkey: "
2474             + Bytes.toStringBinary(key), getTableInfo(), r1);
2475         errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
2476             "Multiple regions have the same startkey: "
2477             + Bytes.toStringBinary(key), getTableInfo(), r2);
2478       }
2479 
2480       @Override
2481       public void handleOverlapInRegionChain(HbckInfo hi1, HbckInfo hi2) throws IOException{
2482         errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
2483             "There is an overlap in the region chain.",
2484             getTableInfo(), hi1, hi2);
2485       }
2486 
2487       @Override
2488       public void handleHoleInRegionChain(byte[] holeStart, byte[] holeStop) throws IOException{
2489         errors.reportError(
2490             ERROR_CODE.HOLE_IN_REGION_CHAIN,
2491             "There is a hole in the region chain between "
2492                 + Bytes.toStringBinary(holeStart) + " and "
2493                 + Bytes.toStringBinary(holeStop)
2494                 + ".  You need to create a new .regioninfo and region "
2495                 + "dir in hdfs to plug the hole.");
2496       }
2497     };
2498 
2499     /**
2500      * This handler fixes integrity errors from hdfs information.  There are
2501      * basically three classes of integrity problems 1) holes, 2) overlaps, and
2502      * 3) invalid regions.
2503      *
2504      * This class overrides methods that fix holes and the overlap group case.
2505      * Individual cases of particular overlaps are handled by the general
2506      * overlap group merge repair case.
2507      *
2508      * If hbase is online, this forces regions offline before doing merge
2509      * operations.
2510      */
2511     private class HDFSIntegrityFixer extends IntegrityFixSuggester {
2512       Configuration conf;
2513 
2514       boolean fixOverlaps = true;
2515 
2516       HDFSIntegrityFixer(TableInfo ti, ErrorReporter errors, Configuration conf,
2517           boolean fixHoles, boolean fixOverlaps) {
2518         super(ti, errors);
2519         this.conf = conf;
2520         this.fixOverlaps = fixOverlaps;
2521         // TODO properly use fixHoles
2522       }
2523 
2524       /**
2525        * This is a special case hole -- when the first region of a table is
2526        * missing from META, HBase doesn't acknowledge the existance of the
2527        * table.
2528        */
2529       @Override
2530       public void handleRegionStartKeyNotEmpty(HbckInfo next) throws IOException {
2531         errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
2532             "First region should start with an empty key.  Creating a new " +
2533             "region and regioninfo in HDFS to plug the hole.",
2534             getTableInfo(), next);
2535         HTableDescriptor htd = getTableInfo().getHTD();
2536         // from special EMPTY_START_ROW to next region's startKey
2537         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(),
2538             HConstants.EMPTY_START_ROW, next.getStartKey());
2539 
2540         // TODO test
2541         HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2542         LOG.info("Table region start key was not empty.  Created new empty region: "
2543             + newRegion + " " +region);
2544         fixes++;
2545       }
2546 
2547       @Override
2548       public void handleRegionEndKeyNotEmpty(byte[] curEndKey) throws IOException {
2549         errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
2550             "Last region should end with an empty key.  Creating a new "
2551                 + "region and regioninfo in HDFS to plug the hole.", getTableInfo());
2552         HTableDescriptor htd = getTableInfo().getHTD();
2553         // from curEndKey to EMPTY_START_ROW
2554         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), curEndKey,
2555             HConstants.EMPTY_START_ROW);
2556 
2557         HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2558         LOG.info("Table region end key was not empty.  Created new empty region: " + newRegion
2559             + " " + region);
2560         fixes++;
2561       }
2562 
2563       /**
2564        * There is a hole in the hdfs regions that violates the table integrity
2565        * rules.  Create a new empty region that patches the hole.
2566        */
2567       @Override
2568       public void handleHoleInRegionChain(byte[] holeStartKey, byte[] holeStopKey) throws IOException {
2569         errors.reportError(
2570             ERROR_CODE.HOLE_IN_REGION_CHAIN,
2571             "There is a hole in the region chain between "
2572                 + Bytes.toStringBinary(holeStartKey) + " and "
2573                 + Bytes.toStringBinary(holeStopKey)
2574                 + ".  Creating a new regioninfo and region "
2575                 + "dir in hdfs to plug the hole.");
2576         HTableDescriptor htd = getTableInfo().getHTD();
2577         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), holeStartKey, holeStopKey);
2578         HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2579         LOG.info("Plugged hole by creating new empty region: "+ newRegion + " " +region);
2580         fixes++;
2581       }
2582 
2583       /**
2584        * This takes set of overlapping regions and merges them into a single
2585        * region.  This covers cases like degenerate regions, shared start key,
2586        * general overlaps, duplicate ranges, and partial overlapping regions.
2587        *
2588        * Cases:
2589        * - Clean regions that overlap
2590        * - Only .oldlogs regions (can't find start/stop range, or figure out)
2591        *
2592        * This is basically threadsafe, except for the fixer increment in mergeOverlaps.
2593        */
2594       @Override
2595       public void handleOverlapGroup(Collection<HbckInfo> overlap)
2596           throws IOException {
2597         Preconditions.checkNotNull(overlap);
2598         Preconditions.checkArgument(overlap.size() >0);
2599 
2600         if (!this.fixOverlaps) {
2601           LOG.warn("Not attempting to repair overlaps.");
2602           return;
2603         }
2604 
2605         if (overlap.size() > maxMerge) {
2606           LOG.warn("Overlap group has " + overlap.size() + " overlapping " +
2607             "regions which is greater than " + maxMerge + ", the max number of regions to merge");
2608           if (sidelineBigOverlaps) {
2609             // we only sideline big overlapped groups that exceeds the max number of regions to merge
2610             sidelineBigOverlaps(overlap);
2611           }
2612           return;
2613         }
2614 
2615         mergeOverlaps(overlap);
2616       }
2617 
2618       void mergeOverlaps(Collection<HbckInfo> overlap)
2619           throws IOException {
2620         String thread = Thread.currentThread().getName();
2621         LOG.info("== [" + thread + "] Merging regions into one region: "
2622           + Joiner.on(",").join(overlap));
2623         // get the min / max range and close all concerned regions
2624         Pair<byte[], byte[]> range = null;
2625         for (HbckInfo hi : overlap) {
2626           if (range == null) {
2627             range = new Pair<byte[], byte[]>(hi.getStartKey(), hi.getEndKey());
2628           } else {
2629             if (RegionSplitCalculator.BYTES_COMPARATOR
2630                 .compare(hi.getStartKey(), range.getFirst()) < 0) {
2631               range.setFirst(hi.getStartKey());
2632             }
2633             if (RegionSplitCalculator.BYTES_COMPARATOR
2634                 .compare(hi.getEndKey(), range.getSecond()) > 0) {
2635               range.setSecond(hi.getEndKey());
2636             }
2637           }
2638           // need to close files so delete can happen.
2639           LOG.debug("[" + thread + "] Closing region before moving data around: " +  hi);
2640           LOG.debug("[" + thread + "] Contained region dir before close");
2641           debugLsr(hi.getHdfsRegionDir());
2642           try {
2643             LOG.info("[" + thread + "] Closing region: " + hi);
2644             closeRegion(hi);
2645           } catch (IOException ioe) {
2646             LOG.warn("[" + thread + "] Was unable to close region " + hi
2647               + ".  Just continuing... ", ioe);
2648           } catch (InterruptedException e) {
2649             LOG.warn("[" + thread + "] Was unable to close region " + hi
2650               + ".  Just continuing... ", e);
2651           }
2652 
2653           try {
2654             LOG.info("[" + thread + "] Offlining region: " + hi);
2655             offline(hi.getRegionName());
2656           } catch (IOException ioe) {
2657             LOG.warn("[" + thread + "] Unable to offline region from master: " + hi
2658               + ".  Just continuing... ", ioe);
2659           }
2660         }
2661 
2662         // create new empty container region.
2663         HTableDescriptor htd = getTableInfo().getHTD();
2664         // from start key to end Key
2665         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), range.getFirst(),
2666             range.getSecond());
2667         HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, htd);
2668         LOG.info("[" + thread + "] Created new empty container region: " +
2669             newRegion + " to contain regions: " + Joiner.on(",").join(overlap));
2670         debugLsr(region.getRegionFileSystem().getRegionDir());
2671 
2672         // all target regions are closed, should be able to safely cleanup.
2673         boolean didFix= false;
2674         Path target = region.getRegionFileSystem().getRegionDir();
2675         for (HbckInfo contained : overlap) {
2676           LOG.info("[" + thread + "] Merging " + contained  + " into " + target );
2677           int merges = mergeRegionDirs(target, contained);
2678           if (merges > 0) {
2679             didFix = true;
2680           }
2681         }
2682         if (didFix) {
2683           fixes++;
2684         }
2685       }
2686 
2687       /**
2688        * Sideline some regions in a big overlap group so that it
2689        * will have fewer regions, and it is easier to merge them later on.
2690        *
2691        * @param bigOverlap the overlapped group with regions more than maxMerge
2692        * @throws IOException
2693        */
2694       void sidelineBigOverlaps(
2695           Collection<HbckInfo> bigOverlap) throws IOException {
2696         int overlapsToSideline = bigOverlap.size() - maxMerge;
2697         if (overlapsToSideline > maxOverlapsToSideline) {
2698           overlapsToSideline = maxOverlapsToSideline;
2699         }
2700         List<HbckInfo> regionsToSideline =
2701           RegionSplitCalculator.findBigRanges(bigOverlap, overlapsToSideline);
2702         FileSystem fs = FileSystem.get(conf);
2703         for (HbckInfo regionToSideline: regionsToSideline) {
2704           try {
2705             LOG.info("Closing region: " + regionToSideline);
2706             closeRegion(regionToSideline);
2707           } catch (IOException ioe) {
2708             LOG.warn("Was unable to close region " + regionToSideline
2709               + ".  Just continuing... ", ioe);
2710           } catch (InterruptedException e) {
2711             LOG.warn("Was unable to close region " + regionToSideline
2712               + ".  Just continuing... ", e);
2713           }
2714 
2715           try {
2716             LOG.info("Offlining region: " + regionToSideline);
2717             offline(regionToSideline.getRegionName());
2718           } catch (IOException ioe) {
2719             LOG.warn("Unable to offline region from master: " + regionToSideline
2720               + ".  Just continuing... ", ioe);
2721           }
2722 
2723           LOG.info("Before sideline big overlapped region: " + regionToSideline.toString());
2724           Path sidelineRegionDir = sidelineRegionDir(fs, TO_BE_LOADED, regionToSideline);
2725           if (sidelineRegionDir != null) {
2726             sidelinedRegions.put(sidelineRegionDir, regionToSideline);
2727             LOG.info("After sidelined big overlapped region: "
2728               + regionToSideline.getRegionNameAsString()
2729               + " to " + sidelineRegionDir.toString());
2730             fixes++;
2731           }
2732         }
2733       }
2734     }
2735 
2736     /**
2737      * Check the region chain (from META) of this table.  We are looking for
2738      * holes, overlaps, and cycles.
2739      * @return false if there are errors
2740      * @throws IOException
2741      */
2742     public boolean checkRegionChain(TableIntegrityErrorHandler handler) throws IOException {
2743       // When table is disabled no need to check for the region chain. Some of the regions
2744       // accidently if deployed, this below code might report some issues like missing start
2745       // or end regions or region hole in chain and may try to fix which is unwanted.
2746       if (disabledTables.contains(this.tableName)) {
2747         return true;
2748       }
2749       int originalErrorsCount = errors.getErrorList().size();
2750       Multimap<byte[], HbckInfo> regions = sc.calcCoverage();
2751       SortedSet<byte[]> splits = sc.getSplits();
2752 
2753       byte[] prevKey = null;
2754       byte[] problemKey = null;
2755 
2756       if (splits.size() == 0) {
2757         // no region for this table
2758         handler.handleHoleInRegionChain(HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
2759       }
2760 
2761       for (byte[] key : splits) {
2762         Collection<HbckInfo> ranges = regions.get(key);
2763         if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) {
2764           for (HbckInfo rng : ranges) {
2765             handler.handleRegionStartKeyNotEmpty(rng);
2766           }
2767         }
2768 
2769         // check for degenerate ranges
2770         for (HbckInfo rng : ranges) {
2771           // special endkey case converts '' to null
2772           byte[] endKey = rng.getEndKey();
2773           endKey = (endKey.length == 0) ? null : endKey;
2774           if (Bytes.equals(rng.getStartKey(),endKey)) {
2775             handler.handleDegenerateRegion(rng);
2776           }
2777         }
2778 
2779         if (ranges.size() == 1) {
2780           // this split key is ok -- no overlap, not a hole.
2781           if (problemKey != null) {
2782             LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
2783           }
2784           problemKey = null; // fell through, no more problem.
2785         } else if (ranges.size() > 1) {
2786           // set the new problem key group name, if already have problem key, just
2787           // keep using it.
2788           if (problemKey == null) {
2789             // only for overlap regions.
2790             LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key));
2791             problemKey = key;
2792           }
2793           overlapGroups.putAll(problemKey, ranges);
2794 
2795           // record errors
2796           ArrayList<HbckInfo> subRange = new ArrayList<HbckInfo>(ranges);
2797           //  this dumb and n^2 but this shouldn't happen often
2798           for (HbckInfo r1 : ranges) {
2799             subRange.remove(r1);
2800             for (HbckInfo r2 : subRange) {
2801               if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) {
2802                 handler.handleDuplicateStartKeys(r1,r2);
2803               } else {
2804                 // overlap
2805                 handler.handleOverlapInRegionChain(r1, r2);
2806               }
2807             }
2808           }
2809 
2810         } else if (ranges.size() == 0) {
2811           if (problemKey != null) {
2812             LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
2813           }
2814           problemKey = null;
2815 
2816           byte[] holeStopKey = sc.getSplits().higher(key);
2817           // if higher key is null we reached the top.
2818           if (holeStopKey != null) {
2819             // hole
2820             handler.handleHoleInRegionChain(key, holeStopKey);
2821           }
2822         }
2823         prevKey = key;
2824       }
2825 
2826       // When the last region of a table is proper and having an empty end key, 'prevKey'
2827       // will be null.
2828       if (prevKey != null) {
2829         handler.handleRegionEndKeyNotEmpty(prevKey);
2830       }
2831 
2832       // TODO fold this into the TableIntegrityHandler
2833       if (getConf().getBoolean("hbasefsck.overlap.merge.parallel", true)) {
2834         boolean ok = handleOverlapsParallel(handler, prevKey);
2835         if (!ok) {
2836           return false;
2837         }
2838       } else {
2839         for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) {
2840           handler.handleOverlapGroup(overlap);
2841         }
2842       }
2843 
2844       if (details) {
2845         // do full region split map dump
2846         errors.print("---- Table '"  +  this.tableName
2847             + "': region split map");
2848         dump(splits, regions);
2849         errors.print("---- Table '"  +  this.tableName
2850             + "': overlap groups");
2851         dumpOverlapProblems(overlapGroups);
2852         errors.print("There are " + overlapGroups.keySet().size()
2853             + " overlap groups with " + overlapGroups.size()
2854             + " overlapping regions");
2855       }
2856       if (!sidelinedRegions.isEmpty()) {
2857         LOG.warn("Sidelined big overlapped regions, please bulk load them!");
2858         errors.print("---- Table '"  +  this.tableName
2859             + "': sidelined big overlapped regions");
2860         dumpSidelinedRegions(sidelinedRegions);
2861       }
2862       return errors.getErrorList().size() == originalErrorsCount;
2863     }
2864 
2865     private boolean handleOverlapsParallel(TableIntegrityErrorHandler handler, byte[] prevKey)
2866         throws IOException {
2867       // we parallelize overlap handler for the case we have lots of groups to fix.  We can
2868       // safely assume each group is independent.
2869       List<WorkItemOverlapMerge> merges = new ArrayList<WorkItemOverlapMerge>(overlapGroups.size());
2870       List<Future<Void>> rets;
2871       for (Collection<HbckInfo> overlap : overlapGroups.asMap().values()) {
2872         //
2873         merges.add(new WorkItemOverlapMerge(overlap, handler));
2874       }
2875       try {
2876         rets = executor.invokeAll(merges);
2877       } catch (InterruptedException e) {
2878         LOG.error("Overlap merges were interrupted", e);
2879         return false;
2880       }
2881       for(int i=0; i<merges.size(); i++) {
2882         WorkItemOverlapMerge work = merges.get(i);
2883         Future<Void> f = rets.get(i);
2884         try {
2885           f.get();
2886         } catch(ExecutionException e) {
2887           LOG.warn("Failed to merge overlap group" + work, e.getCause());
2888         } catch (InterruptedException e) {
2889           LOG.error("Waiting for overlap merges was interrupted", e);
2890           return false;
2891         }
2892       }
2893       return true;
2894     }
2895 
2896     /**
2897      * This dumps data in a visually reasonable way for visual debugging
2898      *
2899      * @param splits
2900      * @param regions
2901      */
2902     void dump(SortedSet<byte[]> splits, Multimap<byte[], HbckInfo> regions) {
2903       // we display this way because the last end key should be displayed as well.
2904       StringBuilder sb = new StringBuilder();
2905       for (byte[] k : splits) {
2906         sb.setLength(0); // clear out existing buffer, if any.
2907         sb.append(Bytes.toStringBinary(k) + ":\t");
2908         for (HbckInfo r : regions.get(k)) {
2909           sb.append("[ "+ r.toString() + ", "
2910               + Bytes.toStringBinary(r.getEndKey())+ "]\t");
2911         }
2912         errors.print(sb.toString());
2913       }
2914     }
2915   }
2916 
2917   public void dumpOverlapProblems(Multimap<byte[], HbckInfo> regions) {
2918     // we display this way because the last end key should be displayed as
2919     // well.
2920     for (byte[] k : regions.keySet()) {
2921       errors.print(Bytes.toStringBinary(k) + ":");
2922       for (HbckInfo r : regions.get(k)) {
2923         errors.print("[ " + r.toString() + ", "
2924             + Bytes.toStringBinary(r.getEndKey()) + "]");
2925       }
2926       errors.print("----");
2927     }
2928   }
2929 
2930   public void dumpSidelinedRegions(Map<Path, HbckInfo> regions) {
2931     for (Map.Entry<Path, HbckInfo> entry: regions.entrySet()) {
2932       TableName tableName = entry.getValue().getTableName();
2933       Path path = entry.getKey();
2934       errors.print("This sidelined region dir should be bulk loaded: "
2935         + path.toString());
2936       errors.print("Bulk load command looks like: "
2937         + "hbase org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles "
2938         + path.toUri().getPath() + " "+ tableName);
2939     }
2940   }
2941 
2942   public Multimap<byte[], HbckInfo> getOverlapGroups(
2943       TableName table) {
2944     TableInfo ti = tablesInfo.get(table);
2945     return ti.overlapGroups;
2946   }
2947 
2948   /**
2949    * Return a list of user-space table names whose metadata have not been
2950    * modified in the last few milliseconds specified by timelag
2951    * if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER, STARTCODE_QUALIFIER,
2952    * SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last
2953    * milliseconds specified by timelag, then the table is a candidate to be returned.
2954    * @return tables that have not been modified recently
2955    * @throws IOException if an error is encountered
2956    */
2957   HTableDescriptor[] getTables(AtomicInteger numSkipped) {
2958     List<TableName> tableNames = new ArrayList<TableName>();
2959     long now = EnvironmentEdgeManager.currentTimeMillis();
2960 
2961     for (HbckInfo hbi : regionInfoMap.values()) {
2962       MetaEntry info = hbi.metaEntry;
2963 
2964       // if the start key is zero, then we have found the first region of a table.
2965       // pick only those tables that were not modified in the last few milliseconds.
2966       if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
2967         if (info.modTime + timelag < now) {
2968           tableNames.add(info.getTable());
2969         } else {
2970           numSkipped.incrementAndGet(); // one more in-flux table
2971         }
2972       }
2973     }
2974     return getHTableDescriptors(tableNames);
2975   }
2976 
2977   HTableDescriptor[] getHTableDescriptors(List<TableName> tableNames) {
2978     HTableDescriptor[] htd = new HTableDescriptor[0];
2979     try {
2980       LOG.info("getHTableDescriptors == tableNames => " + tableNames);
2981       htd = new HBaseAdmin(getConf()).getTableDescriptorsByTableName(tableNames);
2982     } catch (IOException e) {
2983       LOG.debug("Exception getting table descriptors", e);
2984     }
2985     return htd;
2986   }
2987 
2988   /**
2989    * Gets the entry in regionInfo corresponding to the the given encoded
2990    * region name. If the region has not been seen yet, a new entry is added
2991    * and returned.
2992    */
2993   private synchronized HbckInfo getOrCreateInfo(String name) {
2994     HbckInfo hbi = regionInfoMap.get(name);
2995     if (hbi == null) {
2996       hbi = new HbckInfo(null);
2997       regionInfoMap.put(name, hbi);
2998     }
2999     return hbi;
3000   }
3001 
3002   private void checkAndFixTableLocks() throws IOException {
3003     ZooKeeperWatcher zkw = createZooKeeperWatcher();
3004 
3005     try {
3006       TableLockChecker checker = new TableLockChecker(createZooKeeperWatcher(), errors);
3007       checker.checkTableLocks();
3008 
3009       if (this.fixTableLocks) {
3010         checker.fixExpiredTableLocks();
3011       }
3012     } finally {
3013       zkw.close();
3014     }
3015   }
3016 
3017   /**
3018    * Check whether a orphaned table ZNode exists and fix it if requested.
3019    * @throws IOException
3020    * @throws KeeperException
3021    * @throws InterruptedException
3022    */
3023   private void checkAndFixOrphanedTableZNodes()
3024       throws IOException, KeeperException, InterruptedException {
3025     ZooKeeperWatcher zkw = createZooKeeperWatcher();
3026     try {
3027       ZKTable zkTable = new ZKTable(zkw);
3028       Set<TableName> enablingTables = zkTable.getEnablingTables(zkw);
3029       String msg;
3030       TableInfo tableInfo;
3031 
3032       for (TableName tableName : enablingTables) {
3033         // Check whether the table exists in hbase
3034         tableInfo = tablesInfo.get(tableName);
3035         if (tableInfo != null) {
3036           // Table exists.  This table state is in transit.  No problem for this table.
3037           continue;
3038         }
3039 
3040         msg = "Table " + tableName + " not found in hbase:meta. Orphaned table ZNode found.";
3041         LOG.warn(msg);
3042         orphanedTableZNodes.add(tableName);
3043         errors.reportError(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY, msg);
3044       }
3045 
3046       if (orphanedTableZNodes.size() > 0 && this.fixTableZNodes) {
3047         for (TableName tableName : orphanedTableZNodes) {
3048           // Set the table state to be disabled so that if we made mistake, we can trace
3049           // the history and figure it out.
3050           // Another choice is to call checkAndRemoveTableState() to delete the orphaned ZNode.
3051           // Both approaches works.
3052           zkTable.setDisabledTable(tableName);
3053         }
3054       }
3055     } finally {
3056       zkw.close();
3057     }
3058   }
3059 
3060     /**
3061     * Check values in regionInfo for hbase:meta
3062     * Check if zero or more than one regions with hbase:meta are found.
3063     * If there are inconsistencies (i.e. zero or more than one regions
3064     * pretend to be holding the hbase:meta) try to fix that and report an error.
3065     * @throws IOException from HBaseFsckRepair functions
3066    * @throws KeeperException
3067    * @throws InterruptedException
3068     */
3069   boolean checkMetaRegion() throws IOException, KeeperException, InterruptedException {
3070     List<HbckInfo> metaRegions = Lists.newArrayList();
3071     for (HbckInfo value : regionInfoMap.values()) {
3072       if (value.metaEntry != null && value.metaEntry.isMetaRegion()) {
3073         metaRegions.add(value);
3074       }
3075     }
3076 
3077     // There will be always one entry in regionInfoMap corresponding to hbase:meta
3078     // Check the deployed servers. It should be exactly one server.
3079     List<ServerName> servers = new ArrayList<ServerName>();
3080     HbckInfo metaHbckInfo = null;
3081     if (!metaRegions.isEmpty()) {
3082       metaHbckInfo = metaRegions.get(0);
3083       servers = metaHbckInfo.deployedOn;
3084     }
3085     if (servers.size() != 1) {
3086       if (servers.size() == 0) {
3087         errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta is not found on any region.");
3088         if (shouldFixAssignments()) {
3089           errors.print("Trying to fix a problem with hbase:meta..");
3090           setShouldRerun();
3091           // try to fix it (treat it as unassigned region)
3092           HBaseFsckRepair.fixUnassigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
3093           HBaseFsckRepair.waitUntilAssigned(admin, HRegionInfo.FIRST_META_REGIONINFO);
3094         }
3095       } else if (servers.size() > 1) {
3096         errors
3097             .reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta is found on more than one region.");
3098         if (shouldFixAssignments()) {
3099           if (metaHbckInfo == null) {
3100             errors.print(
3101               "Unable to fix problem with hbase:meta due to hbase:meta region info missing");
3102             return false;
3103           }
3104           errors.print("Trying to fix a problem with hbase:meta..");
3105           setShouldRerun();
3106           // try fix it (treat is a dupe assignment)
3107           HBaseFsckRepair.fixMultiAssignment(admin, metaHbckInfo.metaEntry, servers);
3108         }
3109       }
3110       // rerun hbck with hopefully fixed META
3111       return false;
3112     }
3113     // no errors, so continue normally
3114     return true;
3115   }
3116 
3117   /**
3118    * Scan hbase:meta, adding all regions found to the regionInfo map.
3119    * @throws IOException if an error is encountered
3120    */
3121   boolean loadMetaEntries() throws IOException {
3122     MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
3123       int countRecord = 1;
3124 
3125       // comparator to sort KeyValues with latest modtime
3126       final Comparator<Cell> comp = new Comparator<Cell>() {
3127         @Override
3128         public int compare(Cell k1, Cell k2) {
3129           return (int)(k1.getTimestamp() - k2.getTimestamp());
3130         }
3131       };
3132 
3133       @Override
3134       public boolean processRow(Result result) throws IOException {
3135         try {
3136 
3137           // record the latest modification of this META record
3138           long ts =  Collections.max(result.listCells(), comp).getTimestamp();
3139           Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(result);
3140           if (pair == null || pair.getFirst() == null) {
3141             emptyRegionInfoQualifiers.add(result);
3142             errors.reportError(ERROR_CODE.EMPTY_META_CELL,
3143               "Empty REGIONINFO_QUALIFIER found in hbase:meta");
3144             return true;
3145           }
3146           ServerName sn = null;
3147           if (pair.getSecond() != null) {
3148             sn = pair.getSecond();
3149           }
3150           HRegionInfo hri = pair.getFirst();
3151           if (!(isTableIncluded(hri.getTable())
3152               || hri.isMetaRegion())) {
3153             return true;
3154           }
3155           PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(result);
3156           MetaEntry m = new MetaEntry(hri, sn, ts, daughters.getFirst(), daughters.getSecond());
3157           HbckInfo previous = regionInfoMap.get(hri.getEncodedName());
3158           if (previous == null) {
3159             regionInfoMap.put(hri.getEncodedName(), new HbckInfo(m));
3160           } else if (previous.metaEntry == null) {
3161             previous.metaEntry = m;
3162           } else {
3163             throw new IOException("Two entries in hbase:meta are same " + previous);
3164           }
3165           
3166           PairOfSameType<HRegionInfo> mergeRegions = HRegionInfo.getMergeRegions(result);
3167           for (HRegionInfo mergeRegion : new HRegionInfo[] {
3168               mergeRegions.getFirst(), mergeRegions.getSecond() }) {
3169             if (mergeRegion != null) {
3170               // This region is already been merged
3171               HbckInfo hbInfo = getOrCreateInfo(mergeRegion.getEncodedName());
3172               hbInfo.setMerged(true);
3173             }
3174           }
3175 
3176           // show proof of progress to the user, once for every 100 records.
3177           if (countRecord % 100 == 0) {
3178             errors.progress();
3179           }
3180           countRecord++;
3181           return true;
3182         } catch (RuntimeException e) {
3183           LOG.error("Result=" + result);
3184           throw e;
3185         }
3186       }
3187     };
3188     if (!checkMetaOnly) {
3189       // Scan hbase:meta to pick up user regions
3190       MetaScanner.metaScan(getConf(), visitor);
3191     }
3192 
3193     errors.print("");
3194     return true;
3195   }
3196 
3197   /**
3198    * Stores the regioninfo entries scanned from META
3199    */
3200   static class MetaEntry extends HRegionInfo {
3201     ServerName regionServer;   // server hosting this region
3202     long modTime;          // timestamp of most recent modification metadata
3203     HRegionInfo splitA, splitB; //split daughters
3204 
3205     public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime) {
3206       this(rinfo, regionServer, modTime, null, null);
3207     }
3208 
3209     public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime,
3210         HRegionInfo splitA, HRegionInfo splitB) {
3211       super(rinfo);
3212       this.regionServer = regionServer;
3213       this.modTime = modTime;
3214       this.splitA = splitA;
3215       this.splitB = splitB;
3216     }
3217 
3218     @Override
3219     public boolean equals(Object o) {
3220       boolean superEq = super.equals(o);
3221       if (!superEq) {
3222         return superEq;
3223       }
3224 
3225       MetaEntry me = (MetaEntry) o;
3226       if (!regionServer.equals(me.regionServer)) {
3227         return false;
3228       }
3229       return (modTime == me.modTime);
3230     }
3231 
3232     @Override
3233     public int hashCode() {
3234       int hash = Arrays.hashCode(getRegionName());
3235       hash ^= getRegionId();
3236       hash ^= Arrays.hashCode(getStartKey());
3237       hash ^= Arrays.hashCode(getEndKey());
3238       hash ^= Boolean.valueOf(isOffline()).hashCode();
3239       hash ^= getTable().hashCode();
3240       if (regionServer != null) {
3241         hash ^= regionServer.hashCode();
3242       }
3243       hash ^= modTime;
3244       return hash;
3245     }
3246   }
3247 
3248   /**
3249    * Stores the regioninfo entries from HDFS
3250    */
3251   static class HdfsEntry {
3252     HRegionInfo hri;
3253     Path hdfsRegionDir = null;
3254     long hdfsRegionDirModTime  = 0;
3255     boolean hdfsRegioninfoFilePresent = false;
3256     boolean hdfsOnlyEdits = false;
3257   }
3258 
3259   /**
3260    * Stores the regioninfo retrieved from Online region servers.
3261    */
3262   static class OnlineEntry {
3263     HRegionInfo hri;
3264     ServerName hsa;
3265 
3266     @Override
3267     public String toString() {
3268       return hsa.toString() + ";" + hri.getRegionNameAsString();
3269     }
3270   }
3271 
3272   /**
3273    * Maintain information about a particular region.  It gathers information
3274    * from three places -- HDFS, META, and region servers.
3275    */
3276   public static class HbckInfo implements KeyRange {
3277     private MetaEntry metaEntry = null; // info in META
3278     private HdfsEntry hdfsEntry = null; // info in HDFS
3279     private List<OnlineEntry> deployedEntries = Lists.newArrayList(); // on Region Server
3280     private List<ServerName> deployedOn = Lists.newArrayList(); // info on RS's
3281     private boolean skipChecks = false; // whether to skip further checks to this region info.
3282     private boolean isMerged = false;// whether this region has already been merged into another one
3283 
3284     HbckInfo(MetaEntry metaEntry) {
3285       this.metaEntry = metaEntry;
3286     }
3287 
3288     public synchronized void addServer(HRegionInfo hri, ServerName server) {
3289       OnlineEntry rse = new OnlineEntry() ;
3290       rse.hri = hri;
3291       rse.hsa = server;
3292       this.deployedEntries.add(rse);
3293       this.deployedOn.add(server);
3294     }
3295 
3296     @Override
3297     public synchronized String toString() {
3298       StringBuilder sb = new StringBuilder();
3299       sb.append("{ meta => ");
3300       sb.append((metaEntry != null)? metaEntry.getRegionNameAsString() : "null");
3301       sb.append( ", hdfs => " + getHdfsRegionDir());
3302       sb.append( ", deployed => " + Joiner.on(", ").join(deployedEntries));
3303       sb.append(" }");
3304       return sb.toString();
3305     }
3306 
3307     @Override
3308     public byte[] getStartKey() {
3309       if (this.metaEntry != null) {
3310         return this.metaEntry.getStartKey();
3311       } else if (this.hdfsEntry != null) {
3312         return this.hdfsEntry.hri.getStartKey();
3313       } else {
3314         LOG.error("Entry " + this + " has no meta or hdfs region start key.");
3315         return null;
3316       }
3317     }
3318 
3319     @Override
3320     public byte[] getEndKey() {
3321       if (this.metaEntry != null) {
3322         return this.metaEntry.getEndKey();
3323       } else if (this.hdfsEntry != null) {
3324         return this.hdfsEntry.hri.getEndKey();
3325       } else {
3326         LOG.error("Entry " + this + " has no meta or hdfs region start key.");
3327         return null;
3328       }
3329     }
3330 
3331     public TableName getTableName() {
3332       if (this.metaEntry != null) {
3333         return this.metaEntry.getTable();
3334       } else if (this.hdfsEntry != null) {
3335         // we are only guaranteed to have a path and not an HRI for hdfsEntry,
3336         // so we get the name from the Path
3337         Path tableDir = this.hdfsEntry.hdfsRegionDir.getParent();
3338         return FSUtils.getTableName(tableDir);
3339       } else {
3340         // Currently no code exercises this path, but we could add one for
3341         // getting table name from OnlineEntry
3342         return null;
3343       }
3344     }
3345 
3346     public String getRegionNameAsString() {
3347       if (metaEntry != null) {
3348         return metaEntry.getRegionNameAsString();
3349       } else if (hdfsEntry != null) {
3350         if (hdfsEntry.hri != null) {
3351           return hdfsEntry.hri.getRegionNameAsString();
3352         }
3353       }
3354       return null;
3355     }
3356 
3357     public byte[] getRegionName() {
3358       if (metaEntry != null) {
3359         return metaEntry.getRegionName();
3360       } else if (hdfsEntry != null) {
3361         return hdfsEntry.hri.getRegionName();
3362       } else {
3363         return null;
3364       }
3365     }
3366 
3367     Path getHdfsRegionDir() {
3368       if (hdfsEntry == null) {
3369         return null;
3370       }
3371       return hdfsEntry.hdfsRegionDir;
3372     }
3373 
3374     boolean containsOnlyHdfsEdits() {
3375       if (hdfsEntry == null) {
3376         return false;
3377       }
3378       return hdfsEntry.hdfsOnlyEdits;
3379     }
3380 
3381     boolean isHdfsRegioninfoPresent() {
3382       if (hdfsEntry == null) {
3383         return false;
3384       }
3385       return hdfsEntry.hdfsRegioninfoFilePresent;
3386     }
3387 
3388     long getModTime() {
3389       if (hdfsEntry == null) {
3390         return 0;
3391       }
3392       return hdfsEntry.hdfsRegionDirModTime;
3393     }
3394 
3395     HRegionInfo getHdfsHRI() {
3396       if (hdfsEntry == null) {
3397         return null;
3398       }
3399       return hdfsEntry.hri;
3400     }
3401 
3402     public void setSkipChecks(boolean skipChecks) {
3403       this.skipChecks = skipChecks;
3404     }
3405 
3406     public boolean isSkipChecks() {
3407       return skipChecks;
3408     }
3409 
3410     public void setMerged(boolean isMerged) {
3411       this.isMerged = isMerged;
3412     }
3413 
3414     public boolean isMerged() {
3415       return this.isMerged;
3416     }
3417   }
3418 
3419   final static Comparator<HbckInfo> cmp = new Comparator<HbckInfo>() {
3420     @Override
3421     public int compare(HbckInfo l, HbckInfo r) {
3422       if (l == r) {
3423         // same instance
3424         return 0;
3425       }
3426 
3427       int tableCompare = l.getTableName().compareTo(r.getTableName());
3428       if (tableCompare != 0) {
3429         return tableCompare;
3430       }
3431 
3432       int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
3433           l.getStartKey(), r.getStartKey());
3434       if (startComparison != 0) {
3435         return startComparison;
3436       }
3437 
3438       // Special case for absolute endkey
3439       byte[] endKey = r.getEndKey();
3440       endKey = (endKey.length == 0) ? null : endKey;
3441       byte[] endKey2 = l.getEndKey();
3442       endKey2 = (endKey2.length == 0) ? null : endKey2;
3443       int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
3444           endKey2,  endKey);
3445 
3446       if (endComparison != 0) {
3447         return endComparison;
3448       }
3449 
3450       // use regionId as tiebreaker.
3451       // Null is considered after all possible values so make it bigger.
3452       if (l.hdfsEntry == null && r.hdfsEntry == null) {
3453         return 0;
3454       }
3455       if (l.hdfsEntry == null && r.hdfsEntry != null) {
3456         return 1;
3457       }
3458       // l.hdfsEntry must not be null
3459       if (r.hdfsEntry == null) {
3460         return -1;
3461       }
3462       // both l.hdfsEntry and r.hdfsEntry must not be null.
3463       return (int) (l.hdfsEntry.hri.getRegionId()- r.hdfsEntry.hri.getRegionId());
3464     }
3465   };
3466 
3467   /**
3468    * Prints summary of all tables found on the system.
3469    */
3470   private void printTableSummary(SortedMap<TableName, TableInfo> tablesInfo) {
3471     StringBuilder sb = new StringBuilder();
3472     int numOfSkippedRegions;
3473     errors.print("Summary:");
3474     for (TableInfo tInfo : tablesInfo.values()) {
3475       numOfSkippedRegions = (skippedRegions.containsKey(tInfo.getName())) ?
3476           skippedRegions.get(tInfo.getName()).size() : 0;
3477 
3478       if (errors.tableHasErrors(tInfo)) {
3479         errors.print("Table " + tInfo.getName() + " is inconsistent.");
3480       } else if (numOfSkippedRegions > 0){
3481         errors.print("Table " + tInfo.getName() + " is okay (with "
3482           + numOfSkippedRegions + " skipped regions).");
3483       }
3484       else {
3485         errors.print("Table " + tInfo.getName() + " is okay.");
3486       }
3487       errors.print("    Number of regions: " + tInfo.getNumRegions());
3488       if (numOfSkippedRegions > 0) {
3489         Set<String> skippedRegionStrings = skippedRegions.get(tInfo.getName());
3490         System.out.println("    Number of skipped regions: " + numOfSkippedRegions);
3491         System.out.println("      List of skipped regions:");
3492         for(String sr : skippedRegionStrings) {
3493           System.out.println("        " + sr);
3494         }
3495       }
3496       sb.setLength(0); // clear out existing buffer, if any.
3497       sb.append("    Deployed on: ");
3498       for (ServerName server : tInfo.deployedOn) {
3499         sb.append(" " + server.toString());
3500       }
3501       errors.print(sb.toString());
3502     }
3503   }
3504 
3505   static ErrorReporter getErrorReporter(
3506       final Configuration conf) throws ClassNotFoundException {
3507     Class<? extends ErrorReporter> reporter = conf.getClass("hbasefsck.errorreporter", PrintingErrorReporter.class, ErrorReporter.class);
3508     return ReflectionUtils.newInstance(reporter, conf);
3509   }
3510 
3511   public interface ErrorReporter {
3512     enum ERROR_CODE {
3513       UNKNOWN, NO_META_REGION, NULL_META_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
3514       NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
3515       MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
3516       FIRST_REGION_STARTKEY_NOT_EMPTY, LAST_REGION_ENDKEY_NOT_EMPTY, DUPE_STARTKEYS,
3517       HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION,
3518       ORPHAN_HDFS_REGION, LINGERING_SPLIT_PARENT, NO_TABLEINFO_FILE, LINGERING_REFERENCE_HFILE,
3519       WRONG_USAGE, EMPTY_META_CELL, EXPIRED_TABLE_LOCK, ORPHANED_ZK_TABLE_ENTRY, BOUNDARIES_ERROR
3520     }
3521     void clear();
3522     void report(String message);
3523     void reportError(String message);
3524     void reportError(ERROR_CODE errorCode, String message);
3525     void reportError(ERROR_CODE errorCode, String message, TableInfo table);
3526     void reportError(ERROR_CODE errorCode, String message, TableInfo table, HbckInfo info);
3527     void reportError(
3528       ERROR_CODE errorCode,
3529       String message,
3530       TableInfo table,
3531       HbckInfo info1,
3532       HbckInfo info2
3533     );
3534     int summarize();
3535     void detail(String details);
3536     ArrayList<ERROR_CODE> getErrorList();
3537     void progress();
3538     void print(String message);
3539     void resetErrors();
3540     boolean tableHasErrors(TableInfo table);
3541   }
3542 
3543   static class PrintingErrorReporter implements ErrorReporter {
3544     public int errorCount = 0;
3545     private int showProgress;
3546     // How frequently calls to progress() will create output
3547     private static final int progressThreshold = 100;
3548 
3549     Set<TableInfo> errorTables = new HashSet<TableInfo>();
3550 
3551     // for use by unit tests to verify which errors were discovered
3552     private ArrayList<ERROR_CODE> errorList = new ArrayList<ERROR_CODE>();
3553 
3554     @Override
3555     public void clear() {
3556       errorTables.clear();
3557       errorList.clear();
3558       errorCount = 0;
3559     }
3560 
3561     @Override
3562     public synchronized void reportError(ERROR_CODE errorCode, String message) {
3563       if (errorCode == ERROR_CODE.WRONG_USAGE) {
3564         System.err.println(message);
3565         return;
3566       }
3567 
3568       errorList.add(errorCode);
3569       if (!summary) {
3570         System.out.println("ERROR: " + message);
3571       }
3572       errorCount++;
3573       showProgress = 0;
3574     }
3575 
3576     @Override
3577     public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table) {
3578       errorTables.add(table);
3579       reportError(errorCode, message);
3580     }
3581 
3582     @Override
3583     public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table,
3584                                          HbckInfo info) {
3585       errorTables.add(table);
3586       String reference = "(region " + info.getRegionNameAsString() + ")";
3587       reportError(errorCode, reference + " " + message);
3588     }
3589 
3590     @Override
3591     public synchronized void reportError(ERROR_CODE errorCode, String message, TableInfo table,
3592                                          HbckInfo info1, HbckInfo info2) {
3593       errorTables.add(table);
3594       String reference = "(regions " + info1.getRegionNameAsString()
3595           + " and " + info2.getRegionNameAsString() + ")";
3596       reportError(errorCode, reference + " " + message);
3597     }
3598 
3599     @Override
3600     public synchronized void reportError(String message) {
3601       reportError(ERROR_CODE.UNKNOWN, message);
3602     }
3603 
3604     /**
3605      * Report error information, but do not increment the error count.  Intended for cases
3606      * where the actual error would have been reported previously.
3607      * @param message
3608      */
3609     @Override
3610     public synchronized void report(String message) {
3611       if (! summary) {
3612         System.out.println("ERROR: " + message);
3613       }
3614       showProgress = 0;
3615     }
3616 
3617     @Override
3618     public synchronized int summarize() {
3619       System.out.println(Integer.toString(errorCount) +
3620                          " inconsistencies detected.");
3621       if (errorCount == 0) {
3622         System.out.println("Status: OK");
3623         return 0;
3624       } else {
3625         System.out.println("Status: INCONSISTENT");
3626         return -1;
3627       }
3628     }
3629 
3630     @Override
3631     public ArrayList<ERROR_CODE> getErrorList() {
3632       return errorList;
3633     }
3634 
3635     @Override
3636     public synchronized void print(String message) {
3637       if (!summary) {
3638         System.out.println(message);
3639       }
3640     }
3641 
3642     @Override
3643     public boolean tableHasErrors(TableInfo table) {
3644       return errorTables.contains(table);
3645     }
3646 
3647     @Override
3648     public void resetErrors() {
3649       errorCount = 0;
3650     }
3651 
3652     @Override
3653     public synchronized void detail(String message) {
3654       if (details) {
3655         System.out.println(message);
3656       }
3657       showProgress = 0;
3658     }
3659 
3660     @Override
3661     public synchronized void progress() {
3662       if (showProgress++ == progressThreshold) {
3663         if (!summary) {
3664           System.out.print(".");
3665         }
3666         showProgress = 0;
3667       }
3668     }
3669   }
3670 
3671   /**
3672    * Contact a region server and get all information from it
3673    */
3674   static class WorkItemRegion implements Callable<Void> {
3675     private HBaseFsck hbck;
3676     private ServerName rsinfo;
3677     private ErrorReporter errors;
3678     private HConnection connection;
3679 
3680     WorkItemRegion(HBaseFsck hbck, ServerName info,
3681                    ErrorReporter errors, HConnection connection) {
3682       this.hbck = hbck;
3683       this.rsinfo = info;
3684       this.errors = errors;
3685       this.connection = connection;
3686     }
3687 
3688     @Override
3689     public synchronized Void call() throws IOException {
3690       errors.progress();
3691       try {
3692         BlockingInterface server = connection.getAdmin(rsinfo);
3693 
3694         // list all online regions from this region server
3695         List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
3696         regions = filterRegions(regions);
3697 
3698         if (details) {
3699           errors.detail("RegionServer: " + rsinfo.getServerName() +
3700                            " number of regions: " + regions.size());
3701           for (HRegionInfo rinfo: regions) {
3702             errors.detail("  " + rinfo.getRegionNameAsString() +
3703                              " id: " + rinfo.getRegionId() +
3704                              " encoded_name: " + rinfo.getEncodedName() +
3705                              " start: " + Bytes.toStringBinary(rinfo.getStartKey()) +
3706                              " end: " + Bytes.toStringBinary(rinfo.getEndKey()));
3707           }
3708         }
3709 
3710         // check to see if the existence of this region matches the region in META
3711         for (HRegionInfo r:regions) {
3712           HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
3713           hbi.addServer(r, rsinfo);
3714         }
3715       } catch (IOException e) {          // unable to connect to the region server.
3716         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
3717           " Unable to fetch region information. " + e);
3718         throw e;
3719       }
3720       return null;
3721     }
3722 
3723     private List<HRegionInfo> filterRegions(List<HRegionInfo> regions) {
3724       List<HRegionInfo> ret = Lists.newArrayList();
3725       for (HRegionInfo hri : regions) {
3726         if (hri.isMetaTable() || (!hbck.checkMetaOnly
3727             && hbck.isTableIncluded(hri.getTable()))) {
3728           ret.add(hri);
3729         }
3730       }
3731       return ret;
3732     }
3733   }
3734 
3735   /**
3736    * Contact hdfs and get all information about specified table directory into
3737    * regioninfo list.
3738    */
3739   static class WorkItemHdfsDir implements Callable<Void> {
3740     private HBaseFsck hbck;
3741     private FileStatus tableDir;
3742     private ErrorReporter errors;
3743     private FileSystem fs;
3744 
3745     WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors,
3746                     FileStatus status) {
3747       this.hbck = hbck;
3748       this.fs = fs;
3749       this.tableDir = status;
3750       this.errors = errors;
3751     }
3752 
3753     @Override
3754     public synchronized Void call() throws IOException {
3755       try {
3756         // level 2: <HBASE_DIR>/<table>/*
3757         FileStatus[] regionDirs = fs.listStatus(tableDir.getPath());
3758         for (FileStatus regionDir : regionDirs) {
3759           errors.progress();
3760           String encodedName = regionDir.getPath().getName();
3761           // ignore directories that aren't hexadecimal
3762           if (!encodedName.toLowerCase().matches("[0-9a-f]+")) {
3763             continue;
3764           }
3765 
3766           LOG.debug("Loading region info from hdfs:"+ regionDir.getPath());
3767           HbckInfo hbi = hbck.getOrCreateInfo(encodedName);
3768           HdfsEntry he = new HdfsEntry();
3769           synchronized (hbi) {
3770             if (hbi.getHdfsRegionDir() != null) {
3771               errors.print("Directory " + encodedName + " duplicate??" +
3772                            hbi.getHdfsRegionDir());
3773             }
3774 
3775             he.hdfsRegionDir = regionDir.getPath();
3776             he.hdfsRegionDirModTime = regionDir.getModificationTime();
3777             Path regioninfoFile = new Path(he.hdfsRegionDir, HRegionFileSystem.REGION_INFO_FILE);
3778             he.hdfsRegioninfoFilePresent = fs.exists(regioninfoFile);
3779             // we add to orphan list when we attempt to read .regioninfo
3780 
3781             // Set a flag if this region contains only edits
3782             // This is special case if a region is left after split
3783             he.hdfsOnlyEdits = true;
3784             FileStatus[] subDirs = fs.listStatus(regionDir.getPath());
3785             Path ePath = HLogUtil.getRegionDirRecoveredEditsDir(regionDir.getPath());
3786             for (FileStatus subDir : subDirs) {
3787               errors.progress();
3788               String sdName = subDir.getPath().getName();
3789               if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) {
3790                 he.hdfsOnlyEdits = false;
3791                 break;
3792               }
3793             }
3794             hbi.hdfsEntry = he;
3795           }
3796         }
3797       } catch (IOException e) {
3798         // unable to connect to the region server.
3799         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "Table Directory: "
3800             + tableDir.getPath().getName()
3801             + " Unable to fetch region information. " + e);
3802         throw e;
3803       }
3804       return null;
3805     }
3806   }
3807 
3808   /**
3809    * Contact hdfs and get all information about specified table directory into
3810    * regioninfo list.
3811    */
3812   static class WorkItemHdfsRegionInfo implements Callable<Void> {
3813     private HbckInfo hbi;
3814     private HBaseFsck hbck;
3815     private ErrorReporter errors;
3816 
3817     WorkItemHdfsRegionInfo(HbckInfo hbi, HBaseFsck hbck, ErrorReporter errors) {
3818       this.hbi = hbi;
3819       this.hbck = hbck;
3820       this.errors = errors;
3821     }
3822 
3823     @Override
3824     public synchronized Void call() throws IOException {
3825       // only load entries that haven't been loaded yet.
3826       if (hbi.getHdfsHRI() == null) {
3827         try {
3828           errors.progress();
3829           hbck.loadHdfsRegioninfo(hbi);
3830         } catch (IOException ioe) {
3831           String msg = "Orphan region in HDFS: Unable to load .regioninfo from table "
3832               + hbi.getTableName() + " in hdfs dir "
3833               + hbi.getHdfsRegionDir()
3834               + "!  It may be an invalid format or version file.  Treating as "
3835               + "an orphaned regiondir.";
3836           errors.reportError(ERROR_CODE.ORPHAN_HDFS_REGION, msg);
3837           try {
3838             hbck.debugLsr(hbi.getHdfsRegionDir());
3839           } catch (IOException ioe2) {
3840             LOG.error("Unable to read directory " + hbi.getHdfsRegionDir(), ioe2);
3841             throw ioe2;
3842           }
3843           hbck.orphanHdfsDirs.add(hbi);
3844           throw ioe;
3845         }
3846       }
3847       return null;
3848     }
3849   };
3850 
3851   /**
3852    * Display the full report from fsck. This displays all live and dead region
3853    * servers, and all known regions.
3854    */
3855   public static void setDisplayFullReport() {
3856     details = true;
3857   }
3858 
3859   /**
3860    * Set summary mode.
3861    * Print only summary of the tables and status (OK or INCONSISTENT)
3862    */
3863   void setSummary() {
3864     summary = true;
3865   }
3866 
3867   /**
3868    * Set hbase:meta check mode.
3869    * Print only info about hbase:meta table deployment/state
3870    */
3871   void setCheckMetaOnly() {
3872     checkMetaOnly = true;
3873   }
3874 
3875   /**
3876    * Set region boundaries check mode.
3877    */
3878   void setRegionBoundariesCheck() {
3879     checkRegionBoundaries = true;
3880   }
3881 
3882   /**
3883    * Set table locks fix mode.
3884    * Delete table locks held for a long time
3885    */
3886   public void setFixTableLocks(boolean shouldFix) {
3887     fixTableLocks = shouldFix;
3888     fixAny |= shouldFix;
3889   }
3890 
3891   /**
3892    * Set orphaned table ZNodes fix mode.
3893    * Set the table state to disable in the orphaned table ZNode.
3894    */
3895   public void setFixTableZNodes(boolean shouldFix) {
3896     fixTableZNodes = shouldFix;
3897     fixAny |= shouldFix;
3898   }
3899 
3900   /**
3901    * Check if we should rerun fsck again. This checks if we've tried to
3902    * fix something and we should rerun fsck tool again.
3903    * Display the full report from fsck. This displays all live and dead
3904    * region servers, and all known regions.
3905    */
3906   void setShouldRerun() {
3907     rerun = true;
3908   }
3909 
3910   boolean shouldRerun() {
3911     return rerun;
3912   }
3913 
3914   /**
3915    * Fix inconsistencies found by fsck. This should try to fix errors (if any)
3916    * found by fsck utility.
3917    */
3918   public void setFixAssignments(boolean shouldFix) {
3919     fixAssignments = shouldFix;
3920     fixAny |= shouldFix;
3921   }
3922 
3923   boolean shouldFixAssignments() {
3924     return fixAssignments;
3925   }
3926 
3927   public void setFixMeta(boolean shouldFix) {
3928     fixMeta = shouldFix;
3929     fixAny |= shouldFix;
3930   }
3931 
3932   boolean shouldFixMeta() {
3933     return fixMeta;
3934   }
3935 
3936   public void setFixEmptyMetaCells(boolean shouldFix) {
3937     fixEmptyMetaCells = shouldFix;
3938     fixAny |= shouldFix;
3939   }
3940 
3941   boolean shouldFixEmptyMetaCells() {
3942     return fixEmptyMetaCells;
3943   }
3944 
3945   public void setCheckHdfs(boolean checking) {
3946     checkHdfs = checking;
3947   }
3948 
3949   boolean shouldCheckHdfs() {
3950     return checkHdfs;
3951   }
3952 
3953   public void setFixHdfsHoles(boolean shouldFix) {
3954     fixHdfsHoles = shouldFix;
3955     fixAny |= shouldFix;
3956   }
3957 
3958   boolean shouldFixHdfsHoles() {
3959     return fixHdfsHoles;
3960   }
3961 
3962   public void setFixTableOrphans(boolean shouldFix) {
3963     fixTableOrphans = shouldFix;
3964     fixAny |= shouldFix;
3965   }
3966 
3967   boolean shouldFixTableOrphans() {
3968     return fixTableOrphans;
3969   }
3970 
3971   public void setFixHdfsOverlaps(boolean shouldFix) {
3972     fixHdfsOverlaps = shouldFix;
3973     fixAny |= shouldFix;
3974   }
3975 
3976   boolean shouldFixHdfsOverlaps() {
3977     return fixHdfsOverlaps;
3978   }
3979 
3980   public void setFixHdfsOrphans(boolean shouldFix) {
3981     fixHdfsOrphans = shouldFix;
3982     fixAny |= shouldFix;
3983   }
3984 
3985   boolean shouldFixHdfsOrphans() {
3986     return fixHdfsOrphans;
3987   }
3988 
3989   public void setFixVersionFile(boolean shouldFix) {
3990     fixVersionFile = shouldFix;
3991     fixAny |= shouldFix;
3992   }
3993 
3994   public boolean shouldFixVersionFile() {
3995     return fixVersionFile;
3996   }
3997 
3998   public void setSidelineBigOverlaps(boolean sbo) {
3999     this.sidelineBigOverlaps = sbo;
4000   }
4001 
4002   public boolean shouldSidelineBigOverlaps() {
4003     return sidelineBigOverlaps;
4004   }
4005 
4006   public void setFixSplitParents(boolean shouldFix) {
4007     fixSplitParents = shouldFix;
4008     fixAny |= shouldFix;
4009   }
4010 
4011   boolean shouldFixSplitParents() {
4012     return fixSplitParents;
4013   }
4014 
4015   public void setFixReferenceFiles(boolean shouldFix) {
4016     fixReferenceFiles = shouldFix;
4017     fixAny |= shouldFix;
4018   }
4019 
4020   boolean shouldFixReferenceFiles() {
4021     return fixReferenceFiles;
4022   }
4023 
4024   public boolean shouldIgnorePreCheckPermission() {
4025     return !fixAny || ignorePreCheckPermission;
4026   }
4027 
4028   public void setIgnorePreCheckPermission(boolean ignorePreCheckPermission) {
4029     this.ignorePreCheckPermission = ignorePreCheckPermission;
4030   }
4031 
4032   /**
4033    * @param mm maximum number of regions to merge into a single region.
4034    */
4035   public void setMaxMerge(int mm) {
4036     this.maxMerge = mm;
4037   }
4038 
4039   public int getMaxMerge() {
4040     return maxMerge;
4041   }
4042 
4043   public void setMaxOverlapsToSideline(int mo) {
4044     this.maxOverlapsToSideline = mo;
4045   }
4046 
4047   public int getMaxOverlapsToSideline() {
4048     return maxOverlapsToSideline;
4049   }
4050 
4051   /**
4052    * Only check/fix tables specified by the list,
4053    * Empty list means all tables are included.
4054    */
4055   boolean isTableIncluded(TableName table) {
4056     return (tablesIncluded.size() == 0) || tablesIncluded.contains(table);
4057   }
4058 
4059   public void includeTable(TableName table) {
4060     tablesIncluded.add(table);
4061   }
4062 
4063   Set<TableName> getIncludedTables() {
4064     return new HashSet<TableName>(tablesIncluded);
4065   }
4066 
4067   /**
4068    * We are interested in only those tables that have not changed their state in
4069    * hbase:meta during the last few seconds specified by hbase.admin.fsck.timelag
4070    * @param seconds - the time in seconds
4071    */
4072   public void setTimeLag(long seconds) {
4073     timelag = seconds * 1000; // convert to milliseconds
4074   }
4075 
4076   /**
4077    *
4078    * @param sidelineDir - HDFS path to sideline data
4079    */
4080   public void setSidelineDir(String sidelineDir) {
4081     this.sidelineDir = new Path(sidelineDir);
4082   }
4083 
4084   protected HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
4085     return new HFileCorruptionChecker(getConf(), executor, sidelineCorruptHFiles);
4086   }
4087 
4088   public HFileCorruptionChecker getHFilecorruptionChecker() {
4089     return hfcc;
4090   }
4091 
4092   public void setHFileCorruptionChecker(HFileCorruptionChecker hfcc) {
4093     this.hfcc = hfcc;
4094   }
4095 
4096   public void setRetCode(int code) {
4097     this.retcode = code;
4098   }
4099 
4100   public int getRetCode() {
4101     return retcode;
4102   }
4103 
4104   protected HBaseFsck printUsageAndExit() {
4105     StringWriter sw = new StringWriter(2048);
4106     PrintWriter out = new PrintWriter(sw);
4107     out.println("Usage: fsck [opts] {only tables}");
4108     out.println(" where [opts] are:");
4109     out.println("   -help Display help options (this)");
4110     out.println("   -details Display full report of all regions.");
4111     out.println("   -timelag <timeInSeconds>  Process only regions that " +
4112                        " have not experienced any metadata updates in the last " +
4113                        " <timeInSeconds> seconds.");
4114     out.println("   -sleepBeforeRerun <timeInSeconds> Sleep this many seconds" +
4115         " before checking if the fix worked if run with -fix");
4116     out.println("   -summary Print only summary of the tables and status.");
4117     out.println("   -metaonly Only check the state of the hbase:meta table.");
4118     out.println("   -sidelineDir <hdfs://> HDFS path to backup existing meta.");
4119     out.println("   -boundaries Verify that regions boundaries are the same between META and store files.");
4120 
4121     out.println("");
4122     out.println("  Metadata Repair options: (expert features, use with caution!)");
4123     out.println("   -fix              Try to fix region assignments.  This is for backwards compatiblity");
4124     out.println("   -fixAssignments   Try to fix region assignments.  Replaces the old -fix");
4125     out.println("   -fixMeta          Try to fix meta problems.  This assumes HDFS region info is good.");
4126     out.println("   -noHdfsChecking   Don't load/check region info from HDFS."
4127         + " Assumes hbase:meta region info is good. Won't check/fix any HDFS issue, e.g. hole, orphan, or overlap");
4128     out.println("   -fixHdfsHoles     Try to fix region holes in hdfs.");
4129     out.println("   -fixHdfsOrphans   Try to fix region dirs with no .regioninfo file in hdfs");
4130     out.println("   -fixTableOrphans  Try to fix table dirs with no .tableinfo file in hdfs (online mode only)");
4131     out.println("   -fixHdfsOverlaps  Try to fix region overlaps in hdfs.");
4132     out.println("   -fixVersionFile   Try to fix missing hbase.version file in hdfs.");
4133     out.println("   -maxMerge <n>     When fixing region overlaps, allow at most <n> regions to merge. (n=" + DEFAULT_MAX_MERGE +" by default)");
4134     out.println("   -sidelineBigOverlaps  When fixing region overlaps, allow to sideline big overlaps");
4135     out.println("   -maxOverlapsToSideline <n>  When fixing region overlaps, allow at most <n> regions to sideline per group. (n=" + DEFAULT_OVERLAPS_TO_SIDELINE +" by default)");
4136     out.println("   -fixSplitParents  Try to force offline split parents to be online.");
4137     out.println("   -ignorePreCheckPermission  ignore filesystem permission pre-check");
4138     out.println("   -fixReferenceFiles  Try to offline lingering reference store files");
4139     out.println("   -fixEmptyMetaCells  Try to fix hbase:meta entries not referencing any region"
4140         + " (empty REGIONINFO_QUALIFIER rows)");
4141 
4142     out.println("");
4143     out.println("  Datafile Repair options: (expert features, use with caution!)");
4144     out.println("   -checkCorruptHFiles     Check all Hfiles by opening them to make sure they are valid");
4145     out.println("   -sidelineCorruptHFiles  Quarantine corrupted HFiles.  implies -checkCorruptHFiles");
4146 
4147     out.println("");
4148     out.println("  Metadata Repair shortcuts");
4149     out.println("   -repair           Shortcut for -fixAssignments -fixMeta -fixHdfsHoles " +
4150         "-fixHdfsOrphans -fixHdfsOverlaps -fixVersionFile -sidelineBigOverlaps " +
4151         "-fixReferenceFiles -fixTableLocks -fixOrphanedTableZnodes");
4152     out.println("   -repairHoles      Shortcut for -fixAssignments -fixMeta -fixHdfsHoles");
4153 
4154     out.println("");
4155     out.println("  Table lock options");
4156     out.println("   -fixTableLocks    Deletes table locks held for a long time (hbase.table.lock.expire.ms, 10min by default)");
4157 
4158     out.println("");
4159     out.println("  Table Znode options");
4160     out.println("   -fixOrphanedTableZnodes    Set table state in ZNode to disabled if table does not exists");
4161 
4162     out.flush();
4163     errors.reportError(ERROR_CODE.WRONG_USAGE, sw.toString());
4164 
4165     setRetCode(-2);
4166     return this;
4167   }
4168 
4169   /**
4170    * Main program
4171    *
4172    * @param args
4173    * @throws Exception
4174    */
4175   public static void main(String[] args) throws Exception {
4176     // create a fsck object
4177     Configuration conf = HBaseConfiguration.create();
4178     Path hbasedir = FSUtils.getRootDir(conf);
4179     URI defaultFs = hbasedir.getFileSystem(conf).getUri();
4180     FSUtils.setFsDefault(conf, new Path(defaultFs));
4181     int ret = ToolRunner.run(new HBaseFsckTool(conf), args);
4182     System.exit(ret);
4183   }
4184 
4185   /**
4186    * This is a Tool wrapper that gathers -Dxxx=yyy configuration settings from the command line.
4187    */
4188   static class HBaseFsckTool extends Configured implements Tool {
4189     HBaseFsckTool(Configuration conf) { super(conf); }
4190     @Override
4191     public int run(String[] args) throws Exception {
4192       HBaseFsck hbck = new HBaseFsck(getConf());
4193       hbck.exec(hbck.executor, args);
4194       return hbck.getRetCode();
4195     }
4196   };
4197 
4198 
4199   public HBaseFsck exec(ExecutorService exec, String[] args) throws KeeperException, IOException,
4200     ServiceException, InterruptedException {
4201     long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
4202 
4203     boolean checkCorruptHFiles = false;
4204     boolean sidelineCorruptHFiles = false;
4205 
4206     // Process command-line args.
4207     for (int i = 0; i < args.length; i++) {
4208       String cmd = args[i];
4209       if (cmd.equals("-help") || cmd.equals("-h")) {
4210         return printUsageAndExit();
4211       } else if (cmd.equals("-details")) {
4212         setDisplayFullReport();
4213       } else if (cmd.equals("-timelag")) {
4214         if (i == args.length - 1) {
4215           errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -timelag needs a value.");
4216           return printUsageAndExit();
4217         }
4218         try {
4219           long timelag = Long.parseLong(args[i+1]);
4220           setTimeLag(timelag);
4221         } catch (NumberFormatException e) {
4222           errors.reportError(ERROR_CODE.WRONG_USAGE, "-timelag needs a numeric value.");
4223           return printUsageAndExit();
4224         }
4225         i++;
4226       } else if (cmd.equals("-sleepBeforeRerun")) {
4227         if (i == args.length - 1) {
4228           errors.reportError(ERROR_CODE.WRONG_USAGE,
4229             "HBaseFsck: -sleepBeforeRerun needs a value.");
4230           return printUsageAndExit();
4231         }
4232         try {
4233           sleepBeforeRerun = Long.parseLong(args[i+1]);
4234         } catch (NumberFormatException e) {
4235           errors.reportError(ERROR_CODE.WRONG_USAGE, "-sleepBeforeRerun needs a numeric value.");
4236           return printUsageAndExit();
4237         }
4238         i++;
4239       } else if (cmd.equals("-sidelineDir")) {
4240         if (i == args.length - 1) {
4241           errors.reportError(ERROR_CODE.WRONG_USAGE, "HBaseFsck: -sidelineDir needs a value.");
4242           return printUsageAndExit();
4243         }
4244         i++;
4245         setSidelineDir(args[i]);
4246       } else if (cmd.equals("-fix")) {
4247         errors.reportError(ERROR_CODE.WRONG_USAGE,
4248           "This option is deprecated, please use  -fixAssignments instead.");
4249         setFixAssignments(true);
4250       } else if (cmd.equals("-fixAssignments")) {
4251         setFixAssignments(true);
4252       } else if (cmd.equals("-fixMeta")) {
4253         setFixMeta(true);
4254       } else if (cmd.equals("-noHdfsChecking")) {
4255         setCheckHdfs(false);
4256       } else if (cmd.equals("-fixHdfsHoles")) {
4257         setFixHdfsHoles(true);
4258       } else if (cmd.equals("-fixHdfsOrphans")) {
4259         setFixHdfsOrphans(true);
4260       } else if (cmd.equals("-fixTableOrphans")) {
4261         setFixTableOrphans(true);
4262       } else if (cmd.equals("-fixHdfsOverlaps")) {
4263         setFixHdfsOverlaps(true);
4264       } else if (cmd.equals("-fixVersionFile")) {
4265         setFixVersionFile(true);
4266       } else if (cmd.equals("-sidelineBigOverlaps")) {
4267         setSidelineBigOverlaps(true);
4268       } else if (cmd.equals("-fixSplitParents")) {
4269         setFixSplitParents(true);
4270       } else if (cmd.equals("-ignorePreCheckPermission")) {
4271         setIgnorePreCheckPermission(true);
4272       } else if (cmd.equals("-checkCorruptHFiles")) {
4273         checkCorruptHFiles = true;
4274       } else if (cmd.equals("-sidelineCorruptHFiles")) {
4275         sidelineCorruptHFiles = true;
4276       } else if (cmd.equals("-fixReferenceFiles")) {
4277         setFixReferenceFiles(true);
4278       } else if (cmd.equals("-fixEmptyMetaCells")) {
4279         setFixEmptyMetaCells(true);
4280       } else if (cmd.equals("-repair")) {
4281         // this attempts to merge overlapping hdfs regions, needs testing
4282         // under load
4283         setFixHdfsHoles(true);
4284         setFixHdfsOrphans(true);
4285         setFixMeta(true);
4286         setFixAssignments(true);
4287         setFixHdfsOverlaps(true);
4288         setFixVersionFile(true);
4289         setSidelineBigOverlaps(true);
4290         setFixSplitParents(false);
4291         setCheckHdfs(true);
4292         setFixReferenceFiles(true);
4293         setFixTableLocks(true);
4294         setFixTableZNodes(true);
4295       } else if (cmd.equals("-repairHoles")) {
4296         // this will make all missing hdfs regions available but may lose data
4297         setFixHdfsHoles(true);
4298         setFixHdfsOrphans(false);
4299         setFixMeta(true);
4300         setFixAssignments(true);
4301         setFixHdfsOverlaps(false);
4302         setSidelineBigOverlaps(false);
4303         setFixSplitParents(false);
4304         setCheckHdfs(true);
4305       } else if (cmd.equals("-maxOverlapsToSideline")) {
4306         if (i == args.length - 1) {
4307           errors.reportError(ERROR_CODE.WRONG_USAGE,
4308             "-maxOverlapsToSideline needs a numeric value argument.");
4309           return printUsageAndExit();
4310         }
4311         try {
4312           int maxOverlapsToSideline = Integer.parseInt(args[i+1]);
4313           setMaxOverlapsToSideline(maxOverlapsToSideline);
4314         } catch (NumberFormatException e) {
4315           errors.reportError(ERROR_CODE.WRONG_USAGE,
4316             "-maxOverlapsToSideline needs a numeric value argument.");
4317           return printUsageAndExit();
4318         }
4319         i++;
4320       } else if (cmd.equals("-maxMerge")) {
4321         if (i == args.length - 1) {
4322           errors.reportError(ERROR_CODE.WRONG_USAGE,
4323             "-maxMerge needs a numeric value argument.");
4324           return printUsageAndExit();
4325         }
4326         try {
4327           int maxMerge = Integer.parseInt(args[i+1]);
4328           setMaxMerge(maxMerge);
4329         } catch (NumberFormatException e) {
4330           errors.reportError(ERROR_CODE.WRONG_USAGE,
4331             "-maxMerge needs a numeric value argument.");
4332           return printUsageAndExit();
4333         }
4334         i++;
4335       } else if (cmd.equals("-summary")) {
4336         setSummary();
4337       } else if (cmd.equals("-metaonly")) {
4338         setCheckMetaOnly();
4339       } else if (cmd.equals("-boundaries")) {
4340         setRegionBoundariesCheck();
4341       } else if (cmd.equals("-fixTableLocks")) {
4342         setFixTableLocks(true);
4343       } else if (cmd.equals("-fixOrphanedTableZnodes")) {
4344         setFixTableZNodes(true);
4345       } else if (cmd.startsWith("-")) {
4346         errors.reportError(ERROR_CODE.WRONG_USAGE, "Unrecognized option:" + cmd);
4347         return printUsageAndExit();
4348       } else {
4349         includeTable(TableName.valueOf(cmd));
4350         errors.print("Allow checking/fixes for table: " + cmd);
4351       }
4352     }
4353 
4354     errors.print("HBaseFsck command line options: " + StringUtils.join(args, " "));
4355 
4356     // pre-check current user has FS write permission or not
4357     try {
4358       preCheckPermission();
4359     } catch (AccessDeniedException ace) {
4360       Runtime.getRuntime().exit(-1);
4361     } catch (IOException ioe) {
4362       Runtime.getRuntime().exit(-1);
4363     }
4364 
4365     // do the real work of hbck
4366     connect();
4367 
4368     try {
4369       // if corrupt file mode is on, first fix them since they may be opened later
4370       if (checkCorruptHFiles || sidelineCorruptHFiles) {
4371         LOG.info("Checking all hfiles for corruption");
4372         HFileCorruptionChecker hfcc = createHFileCorruptionChecker(sidelineCorruptHFiles);
4373         setHFileCorruptionChecker(hfcc); // so we can get result
4374         Collection<TableName> tables = getIncludedTables();
4375         Collection<Path> tableDirs = new ArrayList<Path>();
4376         Path rootdir = FSUtils.getRootDir(getConf());
4377         if (tables.size() > 0) {
4378           for (TableName t : tables) {
4379             tableDirs.add(FSUtils.getTableDir(rootdir, t));
4380           }
4381         } else {
4382           tableDirs = FSUtils.getTableDirs(FSUtils.getCurrentFileSystem(getConf()), rootdir);
4383         }
4384         hfcc.checkTables(tableDirs);
4385         hfcc.report(errors);
4386       }
4387 
4388       // check and fix table integrity, region consistency.
4389       int code = onlineHbck();
4390       setRetCode(code);
4391       // If we have changed the HBase state it is better to run hbck again
4392       // to see if we haven't broken something else in the process.
4393       // We run it only once more because otherwise we can easily fall into
4394       // an infinite loop.
4395       if (shouldRerun()) {
4396         try {
4397           LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix...");
4398           Thread.sleep(sleepBeforeRerun);
4399         } catch (InterruptedException ie) {
4400           return this;
4401         }
4402         // Just report
4403         setFixAssignments(false);
4404         setFixMeta(false);
4405         setFixHdfsHoles(false);
4406         setFixHdfsOverlaps(false);
4407         setFixVersionFile(false);
4408         setFixTableOrphans(false);
4409         errors.resetErrors();
4410         code = onlineHbck();
4411         setRetCode(code);
4412       }
4413     } finally {
4414       IOUtils.cleanup(null, connection, meta, admin);
4415     }
4416     return this;
4417   }
4418 
4419   /**
4420    * ls -r for debugging purposes
4421    */
4422   void debugLsr(Path p) throws IOException {
4423     debugLsr(getConf(), p, errors);
4424   }
4425 
4426   /**
4427    * ls -r for debugging purposes
4428    */
4429   public static void debugLsr(Configuration conf,
4430       Path p) throws IOException {
4431     debugLsr(conf, p, new PrintingErrorReporter());
4432   }
4433 
4434   /**
4435    * ls -r for debugging purposes
4436    */
4437   public static void debugLsr(Configuration conf,
4438       Path p, ErrorReporter errors) throws IOException {
4439     if (!LOG.isDebugEnabled() || p == null) {
4440       return;
4441     }
4442     FileSystem fs = p.getFileSystem(conf);
4443 
4444     if (!fs.exists(p)) {
4445       // nothing
4446       return;
4447     }
4448     errors.print(p.toString());
4449 
4450     if (fs.isFile(p)) {
4451       return;
4452     }
4453 
4454     if (fs.getFileStatus(p).isDir()) {
4455       FileStatus[] fss= fs.listStatus(p);
4456       for (FileStatus status : fss) {
4457         debugLsr(conf, status.getPath(), errors);
4458       }
4459     }
4460   }
4461 }