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