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