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