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