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