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