View Javadoc

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