View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.util;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.Collections;
26  import java.util.Comparator;
27  import java.util.HashSet;
28  import java.util.List;
29  import java.util.Map.Entry;
30  import java.util.Set;
31  import java.util.SortedSet;
32  import java.util.TreeMap;
33  import java.util.TreeSet;
34  import java.util.concurrent.LinkedBlockingQueue;
35  import java.util.concurrent.ThreadPoolExecutor;
36  import java.util.concurrent.TimeUnit;
37  import java.util.concurrent.atomic.AtomicInteger;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FSDataInputStream;
43  import org.apache.hadoop.fs.FileStatus;
44  import org.apache.hadoop.fs.FileSystem;
45  import org.apache.hadoop.fs.Path;
46  import org.apache.hadoop.hbase.Abortable;
47  import org.apache.hadoop.hbase.ClusterStatus;
48  import org.apache.hadoop.hbase.HBaseConfiguration;
49  import org.apache.hadoop.hbase.HConstants;
50  import org.apache.hadoop.hbase.HRegionInfo;
51  import org.apache.hadoop.hbase.HRegionLocation;
52  import org.apache.hadoop.hbase.HServerAddress;
53  import org.apache.hadoop.hbase.HTableDescriptor;
54  import org.apache.hadoop.hbase.KeyValue;
55  import org.apache.hadoop.hbase.MasterNotRunningException;
56  import org.apache.hadoop.hbase.ServerName;
57  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.client.HBaseAdmin;
60  import org.apache.hadoop.hbase.client.HConnection;
61  import org.apache.hadoop.hbase.client.HConnectionManager;
62  import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
63  import org.apache.hadoop.hbase.client.MetaScanner;
64  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
65  import org.apache.hadoop.hbase.client.Put;
66  import org.apache.hadoop.hbase.client.Result;
67  import org.apache.hadoop.hbase.ipc.HRegionInterface;
68  import org.apache.hadoop.hbase.master.MasterFileSystem;
69  import org.apache.hadoop.hbase.regionserver.HRegion;
70  import org.apache.hadoop.hbase.regionserver.wal.HLog;
71  import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
72  import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
73  import org.apache.hadoop.hbase.zookeeper.ZKTable;
74  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
75  import org.apache.hadoop.io.MultipleIOException;
76  import org.apache.zookeeper.KeeperException;
77  
78  import com.google.common.base.Joiner;
79  import com.google.common.collect.Lists;
80  import com.google.common.collect.Multimap;
81  import com.google.common.collect.TreeMultimap;
82  
83  /**
84   * Check consistency among the in-memory states of the master and the
85   * region server(s) and the state of data in HDFS.
86   */
87  public class HBaseFsck {
88    public static final long DEFAULT_TIME_LAG = 60000; // default value of 1 minute
89    public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000;
90  
91    private static final int MAX_NUM_THREADS = 50; // #threads to contact regions
92    private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
93  
94    private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
95    private Configuration conf;
96  
97    private ClusterStatus status;
98    private HConnection connection;
99  
100   private TreeMap<String, HbckInfo> regionInfo = new TreeMap<String, HbckInfo>();
101   private TreeMap<String, TInfo> tablesInfo = new TreeMap<String, TInfo>();
102   private TreeSet<byte[]> disabledTables =
103     new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
104   ErrorReporter errors = new PrintingErrorReporter();
105 
106   private static boolean details = false; // do we display the full report
107   private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
108   private boolean fix = false; // do we want to try fixing the errors?
109   private boolean rerun = false; // if we tried to fix something rerun hbck
110   private static boolean summary = false; // if we want to print less output
111   private boolean checkMetaOnly = false;
112   
113   // Empty regioninfo qualifiers in .META.
114   private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
115   private int numThreads = MAX_NUM_THREADS;
116   private HBaseAdmin admin;
117 
118   ThreadPoolExecutor executor; // threads to retrieve data from regionservers
119 
120   /**
121    * Constructor
122    *
123    * @param conf Configuration object
124    * @throws MasterNotRunningException if the master is not running
125    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
126    */
127   public HBaseFsck(Configuration conf) throws MasterNotRunningException,
128       ZooKeeperConnectionException, IOException {
129     this.conf = conf;
130 
131     numThreads = conf.getInt("hbasefsck.numthreads", numThreads);
132     executor = new ThreadPoolExecutor(0, numThreads,
133         THREADS_KEEP_ALIVE_SECONDS, TimeUnit.SECONDS,
134         new LinkedBlockingQueue<Runnable>());
135   }
136 
137   public void connect() throws MasterNotRunningException,
138       ZooKeeperConnectionException {
139     admin = new HBaseAdmin(conf);
140     status = admin.getMaster().getClusterStatus();
141     connection = admin.getConnection();
142   }
143 
144   /**
145    * Contacts the master and prints out cluster-wide information
146    * @throws IOException if a remote or network exception occurs
147    * @return 0 on success, non-zero on failure
148    * @throws KeeperException
149    * @throws InterruptedException
150    */
151   public int doWork() throws IOException, KeeperException, InterruptedException {
152     // print hbase server version
153     errors.print("Version: " + status.getHBaseVersion());
154 
155     // Make sure regionInfo is empty before starting
156     regionInfo.clear();
157     tablesInfo.clear();
158     emptyRegionInfoQualifiers.clear();
159     disabledTables.clear();
160     errors.clear();
161 
162     // get a list of all regions from the master. This involves
163     // scanning the META table
164     if (!recordRootRegion()) {
165       // Will remove later if we can fix it
166       errors.reportError("Encountered fatal error. Exiting...");
167       return -1;
168     }
169     
170     getMetaEntries();
171 
172     // Check if .META. is found only once and in the right place
173     if (!checkMetaEntries()) {
174       // Will remove later if we can fix it
175       errors.reportError("Encountered fatal error. Exiting...");
176       return -1;
177     }
178 
179     // get a list of all tables that have not changed recently.
180     if (!checkMetaOnly) {
181       AtomicInteger numSkipped = new AtomicInteger(0);
182       HTableDescriptor[] allTables = getTables(numSkipped);
183       errors.print("Number of Tables: " +
184           (allTables == null ? 0 : allTables.length));
185       if (details) {
186         if (numSkipped.get() > 0) {
187           errors.detail("Number of Tables in flux: " + numSkipped.get());
188         }
189         if (allTables != null && allTables.length > 0) {
190           for (HTableDescriptor td : allTables) {
191           String tableName = td.getNameAsString();
192           errors.detail("  Table: " + tableName + "\t" +
193                              (td.isReadOnly() ? "ro" : "rw") + "\t" +
194                              (td.isRootRegion() ? "ROOT" :
195                               (td.isMetaRegion() ? "META" : "    ")) + "\t" +
196                              " families: " + td.getFamilies().size());
197           }
198         }
199       }
200     }
201     
202     // From the master, get a list of all known live region servers
203     Collection<ServerName> regionServers = status.getServers();
204     errors.print("Number of live region servers: " +
205                        regionServers.size());
206     if (details) {
207       for (ServerName rsinfo: regionServers) {
208         errors.print("  " + rsinfo);
209       }
210     }
211 
212     // From the master, get a list of all dead region servers
213     Collection<ServerName> deadRegionServers = status.getDeadServerNames();
214     errors.print("Number of dead region servers: " +
215                        deadRegionServers.size());
216     if (details) {
217       for (ServerName name: deadRegionServers) {
218         errors.print("  " + name);
219       }
220     }
221 
222     // Determine what's deployed
223     processRegionServers(regionServers);
224 
225     // Determine what's on HDFS
226     checkHdfs();
227 
228     // Empty cells in .META.?
229     errors.print("Number of empty REGIONINFO_QUALIFIER rows in .META.: " +
230       emptyRegionInfoQualifiers.size());
231     if (details) {
232       for (Result r: emptyRegionInfoQualifiers) {
233         errors.print("  " + r);
234       }
235     }
236 
237     // Get disabled tables from ZooKeeper
238     loadDisabledTables();
239 
240     // Check consistency
241     checkConsistency();
242 
243     // Check integrity
244     checkIntegrity();
245 
246     // Print table summary
247     printTableSummary();
248 
249     return errors.summarize();
250   }
251 
252   public ErrorReporter getErrors() {
253     return errors;
254   }
255 
256   /**
257    * Populate a specific hbi from regioninfo on file system.
258    */
259   private void loadMetaEntry(HbckInfo hbi) throws IOException {
260     Path regionDir = hbi.foundRegionDir.getPath();
261     Path regioninfo = new Path(regionDir, HRegion.REGIONINFO_FILE);
262     FileSystem fs = FileSystem.get(conf);
263     FSDataInputStream in = fs.open(regioninfo);
264     byte[] tableName = Bytes.toBytes(hbi.hdfsTableName);
265     HRegionInfo hri = new HRegionInfo(tableName);
266     hri.readFields(in);
267     in.close();
268     LOG.debug("HRegionInfo read: " + hri.toString());
269     hbi.metaEntry = new MetaEntry(hri, null,
270         hbi.foundRegionDir.getModificationTime());
271   }
272 
273   public static class RegionInfoLoadException extends IOException {
274     private static final long serialVersionUID = 1L;
275     final IOException ioe;
276     public RegionInfoLoadException(String s, IOException ioe) {
277       super(s);
278       this.ioe = ioe;
279     }
280   }
281 
282   /**
283    * Populate hbi's from regionInfos loaded from file system. 
284    */
285   private void loadTableInfo() throws IOException {
286     List<IOException> ioes = new ArrayList<IOException>();
287     // generate region split structure
288     for (HbckInfo hbi : regionInfo.values()) {
289       // only load entries that haven't been loaded yet.
290       if (hbi.metaEntry == null) {
291         try {
292           loadMetaEntry(hbi);
293         } catch (IOException ioe) {
294           String msg = "Unable to load region info for table " + hbi.hdfsTableName
295             + "!  It may be an invalid format or version file.  You may want to "
296             + "remove " + hbi.foundRegionDir.getPath()
297             + " region from hdfs and retry.";
298           errors.report(msg);
299           LOG.error(msg, ioe);
300           ioes.add(new RegionInfoLoadException(msg, ioe));
301           continue;
302         }
303       }
304 
305       // get table name from hdfs, populate various HBaseFsck tables.
306       String tableName = hbi.hdfsTableName;
307       TInfo modTInfo = tablesInfo.get(tableName);
308       if (modTInfo == null) {
309         modTInfo = new TInfo(tableName);
310       }
311       modTInfo.addRegionInfo(hbi);
312       tablesInfo.put(tableName, modTInfo);
313     }
314 
315     if (ioes.size() != 0) {
316       throw MultipleIOException.createIOException(ioes);
317     }
318   }
319 
320   /**
321    * This borrows code from MasterFileSystem.bootstrap()
322    * 
323    * @return an open .META. HRegion
324    */
325   private HRegion createNewRootAndMeta() throws IOException {
326     Path rootdir = new Path(conf.get(HConstants.HBASE_DIR));
327     Configuration c = conf;
328     HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
329     MasterFileSystem.setInfoFamilyCachingForRoot(false);
330     HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
331     MasterFileSystem.setInfoFamilyCachingForMeta(false);
332     HRegion root = HRegion.createHRegion(rootHRI, rootdir, c,
333         HTableDescriptor.ROOT_TABLEDESC);
334     HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c,
335         HTableDescriptor.META_TABLEDESC);
336     MasterFileSystem.setInfoFamilyCachingForRoot(true);
337     MasterFileSystem.setInfoFamilyCachingForMeta(true);
338 
339     // Add first region from the META table to the ROOT region.
340     HRegion.addRegionToMETA(root, meta);
341     root.close();
342     root.getLog().closeAndDelete();
343     return meta;
344   }
345 
346   /**
347    * Generate set of puts to add to new meta.  This expects the tables to be 
348    * clean with no overlaps or holes.  If there are any problems it returns null.
349    * 
350    * @return An array list of puts to do in bulk, null if tables have problems
351    */
352   private ArrayList<Put> generatePuts() throws IOException {
353     ArrayList<Put> puts = new ArrayList<Put>();
354     boolean hasProblems = false;
355     for (Entry<String, TInfo> e : tablesInfo.entrySet()) {
356       String name = e.getKey();
357 
358       // skip "-ROOT-" and ".META."
359       if (Bytes.compareTo(Bytes.toBytes(name), HConstants.ROOT_TABLE_NAME) == 0
360           || Bytes.compareTo(Bytes.toBytes(name), HConstants.META_TABLE_NAME) == 0) {
361         continue;
362       }
363 
364       TInfo ti = e.getValue();
365       for (Entry<byte[], Collection<HbckInfo>> spl : ti.sc.getStarts().asMap()
366           .entrySet()) {
367         Collection<HbckInfo> his = spl.getValue();
368         int sz = his.size();
369         if (sz != 1) {
370           // problem
371           LOG.error("Split starting at " + Bytes.toStringBinary(spl.getKey())
372               + " had " +  sz + " regions instead of exactly 1." );
373           hasProblems = true;
374           continue;
375         }
376 
377         // add the row directly to meta.
378         HbckInfo hi = his.iterator().next();
379         HRegionInfo hri = hi.metaEntry;
380         Put p = new Put(hri.getRegionName());
381         p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
382             Writables.getBytes(hri));
383         puts.add(p);
384       }
385     }
386     return hasProblems ? null : puts;
387   }
388 
389   /**
390    * Suggest fixes for each table
391    */
392   private void suggestFixes(TreeMap<String, TInfo> tablesInfo) {
393     for (TInfo tInfo : tablesInfo.values()) {
394       tInfo.checkRegionChain();
395     }
396   }
397 
398 
399   /**
400    * Rebuilds meta from information in hdfs/fs.  Depends on configuration
401    * settings passed into hbck constructor to point to a particular fs/dir.
402    * 
403    * @return true if successful, false if attempt failed.
404    */
405   public boolean rebuildMeta() throws IOException, InterruptedException {
406     // TODO check to make sure hbase is offline. (or at least the table
407     // currently being worked on is off line)
408 
409     // Determine what's on HDFS
410     LOG.info("Loading HBase regioninfo from HDFS...");
411     checkHdfs(); // populating regioninfo table.
412     loadTableInfo(); // update tableInfos based on region info in fs.
413 
414     LOG.info("Checking HBase region split map from HDFS data...");
415     int errs = errors.getErrorList().size();
416     for (TInfo tInfo : tablesInfo.values()) {
417       if (!tInfo.checkRegionChain()) {
418         // should dump info as well.
419         errors.report("Found inconsistency in table " + tInfo.getName());
420       }
421     }
422 
423     // make sure ok.
424     if (errors.getErrorList().size() != errs) {
425       suggestFixes(tablesInfo);
426 
427       // Not ok, bail out.
428       return false;
429     }
430 
431     // we can rebuild, move old root and meta out of the way and start
432     LOG.info("HDFS regioninfo's seems good.  Sidelining old .META.");
433     sidelineOldRootAndMeta();
434     
435     LOG.info("Creating new .META.");
436     HRegion meta = createNewRootAndMeta();
437 
438     // populate meta
439     List<Put> puts = generatePuts();
440     if (puts == null) {
441       LOG.fatal("Problem encountered when creating new .META. entries.  " +
442         "You may need to restore the previously sidlined -ROOT- and .META.");
443       return false;
444     }
445     meta.put(puts.toArray(new Put[0]));
446     meta.close();
447     meta.getLog().closeAndDelete();
448     LOG.info("Success! .META. table rebuilt.");
449     return true;
450   }
451 
452   void sidelineTable(FileSystem fs, byte[] table, Path hbaseDir, 
453       Path backupHbaseDir) throws IOException {
454     String tableName = Bytes.toString(table);
455     Path tableDir = new Path(hbaseDir, tableName);
456     if (fs.exists(tableDir)) {
457       Path backupTableDir= new Path(backupHbaseDir, tableName);
458       boolean success = fs.rename(tableDir, backupTableDir); 
459       if (!success) {
460         throw new IOException("Failed to move  " + tableName + " from " 
461             +  tableDir.getName() + " to " + backupTableDir.getName());
462       }
463     } else {
464       LOG.info("No previous " + tableName +  " exists.  Continuing.");
465     }
466   }
467   
468   /**
469    * @return Path to backup of original directory
470    * @throws IOException
471    */
472   Path sidelineOldRootAndMeta() throws IOException {
473     // put current -ROOT- and .META. aside.
474     Path hbaseDir = new Path(conf.get(HConstants.HBASE_DIR));
475     FileSystem fs = hbaseDir.getFileSystem(conf);
476     long now = System.currentTimeMillis();
477     Path backupDir = new Path(hbaseDir.getParent(), hbaseDir.getName() + "-"
478         + now);
479     fs.mkdirs(backupDir);
480 
481     sidelineTable(fs, HConstants.ROOT_TABLE_NAME, hbaseDir, backupDir);
482     try {
483       sidelineTable(fs, HConstants.META_TABLE_NAME, hbaseDir, backupDir);
484     } catch (IOException e) {
485       LOG.error("Attempt to sideline meta failed, attempt to revert...", e);
486       try {
487         // move it back.
488         sidelineTable(fs, HConstants.ROOT_TABLE_NAME, backupDir, hbaseDir);
489         LOG.warn("... revert succeed.  -ROOT- and .META. still in "
490             + "original state.");
491       } catch (IOException ioe) {
492         LOG.fatal("... failed to sideline root and meta and failed to restore "
493             + "prevoius state.  Currently in inconsistent state.  To restore "
494             + "try to rename -ROOT- in " + backupDir.getName() + " to " 
495             + hbaseDir.getName() + ".", ioe);
496       }
497       throw e; // throw original exception
498     }
499     return backupDir;
500   }
501 
502   /**
503    * Load the list of disabled tables in ZK into local set.
504    * @throws ZooKeeperConnectionException
505    * @throws IOException
506    */
507   private void loadDisabledTables()
508   throws ZooKeeperConnectionException, IOException {
509     HConnectionManager.execute(new HConnectable<Void>(conf) {
510       @Override
511       public Void connect(HConnection connection) throws IOException {
512         ZooKeeperWatcher zkw = connection.getZooKeeperWatcher();
513         try {
514           for (String tableName : ZKTable.getDisabledOrDisablingTables(zkw)) {
515             disabledTables.add(Bytes.toBytes(tableName));
516           }
517         } catch (KeeperException ke) {
518           throw new IOException(ke);
519         }
520         return null;
521       }
522     });
523   }
524 
525   /**
526    * Check if the specified region's table is disabled.
527    * @throws ZooKeeperConnectionException
528    * @throws IOException
529    * @throws KeeperException
530    */
531   private boolean isTableDisabled(HRegionInfo regionInfo) {
532     return disabledTables.contains(regionInfo.getTableName());
533   }
534 
535   /**
536    * Scan HDFS for all regions, recording their information into
537    * regionInfo
538    */
539   public void checkHdfs() throws IOException, InterruptedException {
540     Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
541     FileSystem fs = rootDir.getFileSystem(conf);
542 
543     // list all tables from HDFS
544     List<FileStatus> tableDirs = Lists.newArrayList();
545 
546     boolean foundVersionFile = false;
547     FileStatus[] files = fs.listStatus(rootDir);
548     for (FileStatus file : files) {
549       String dirName = file.getPath().getName();
550       if (dirName.equals(HConstants.VERSION_FILE_NAME)) {
551         foundVersionFile = true;
552       } else {
553         if (!checkMetaOnly ||
554             dirName.equals("-ROOT-") ||
555             dirName.equals(".META.")) {
556           tableDirs.add(file);
557         }
558       }
559     }
560 
561     // verify that version file exists
562     if (!foundVersionFile) {
563       errors.reportError(ERROR_CODE.NO_VERSION_FILE,
564           "Version file does not exist in root dir " + rootDir);
565     }
566 
567     // level 1:  <HBASE_DIR>/*
568     WorkItemHdfsDir[] dirs = new WorkItemHdfsDir[tableDirs.size()];  
569     int num = 0;
570     for (FileStatus tableDir : tableDirs) {
571       dirs[num] = new WorkItemHdfsDir(this, fs, errors, tableDir); 
572       executor.execute(dirs[num]);
573       num++;
574     }
575 
576     // wait for all directories to be done
577     for (int i = 0; i < num; i++) {
578       synchronized (dirs[i]) {
579         while (!dirs[i].isDone()) {
580           dirs[i].wait();
581         }
582       }
583     }
584   }
585 
586   /**
587    * Record the location of the ROOT region as found in ZooKeeper,
588    * as if it were in a META table. This is so that we can check
589    * deployment of ROOT.
590    */
591   boolean recordRootRegion() throws IOException {
592     HRegionLocation rootLocation = connection.locateRegion(
593       HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW);
594 
595     // Check if Root region is valid and existing
596     if (rootLocation == null || rootLocation.getRegionInfo() == null ||
597         rootLocation.getHostname() == null) {
598       errors.reportError(ERROR_CODE.NULL_ROOT_REGION,
599         "Root Region or some of its attributes are null.");
600       return false;
601     }
602     ServerName sn;
603     try {
604       sn = getRootRegionServerName();
605     } catch (InterruptedException e) {
606       throw new IOException("Interrupted", e);
607     }
608     MetaEntry m =
609       new MetaEntry(rootLocation.getRegionInfo(), sn, System.currentTimeMillis());
610     HbckInfo hbInfo = new HbckInfo(m);
611     regionInfo.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
612     return true;
613   }
614 
615   private ServerName getRootRegionServerName()
616   throws IOException, InterruptedException {
617     RootRegionTracker rootRegionTracker =
618       new RootRegionTracker(this.connection.getZooKeeperWatcher(), new Abortable() {
619         @Override
620         public void abort(String why, Throwable e) {
621           LOG.error(why, e);
622           System.exit(1);
623         }
624         @Override
625         public boolean isAborted(){
626           return false;
627         }
628         
629       });
630     rootRegionTracker.start();
631     ServerName sn = null;
632     try {
633       sn = rootRegionTracker.getRootRegionLocation();
634     } finally {
635       rootRegionTracker.stop();
636     }
637     return sn;
638   }
639 
640   /**
641    * Contacts each regionserver and fetches metadata about regions.
642    * @param regionServerList - the list of region servers to connect to
643    * @throws IOException if a remote or network exception occurs
644    */
645   void processRegionServers(Collection<ServerName> regionServerList)
646   throws IOException, InterruptedException {
647     WorkItemRegion[] work = new WorkItemRegion[regionServerList.size()];
648     int num = 0;
649 
650     // loop to contact each region server in parallel
651     for (ServerName rsinfo: regionServerList) {
652       work[num] = new WorkItemRegion(this, rsinfo, errors, connection);
653       executor.execute(work[num]);
654       num++;
655     }
656     
657     // wait for all submitted tasks to be done
658     for (int i = 0; i < num; i++) {
659       synchronized (work[i]) {
660         while (!work[i].isDone()) {
661           work[i].wait();
662         }
663       }
664     }
665   }
666 
667   /**
668    * Check consistency of all regions that have been found in previous phases.
669    * @throws KeeperException
670    * @throws InterruptedException
671    */
672   void checkConsistency()
673   throws IOException, KeeperException, InterruptedException {
674     for (java.util.Map.Entry<String, HbckInfo> e: regionInfo.entrySet()) {
675       doConsistencyCheck(e.getKey(), e.getValue());
676     }
677   }
678 
679   /**
680    * Check a single region for consistency and correct deployment.
681    * @throws KeeperException
682    * @throws InterruptedException
683    */
684   void doConsistencyCheck(final String key, final HbckInfo hbi)
685   throws IOException, KeeperException, InterruptedException {
686     String descriptiveName = hbi.toString();
687 
688     boolean inMeta = hbi.metaEntry != null;
689     boolean inHdfs = hbi.foundRegionDir != null;
690     boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null;
691     boolean isDeployed = !hbi.deployedOn.isEmpty();
692     boolean isMultiplyDeployed = hbi.deployedOn.size() > 1;
693     boolean deploymentMatchesMeta =
694       hasMetaAssignment && isDeployed && !isMultiplyDeployed &&
695       hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0));
696     boolean splitParent =
697       (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
698     boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
699     boolean recentlyModified = hbi.foundRegionDir != null &&
700       hbi.foundRegionDir.getModificationTime() + timelag > System.currentTimeMillis();
701 
702     // ========== First the healthy cases =============
703     if (hbi.onlyEdits) {
704       return;
705     }
706     if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) {
707       return;
708     } else if (inMeta && !isDeployed && splitParent) {
709       return;
710     } else if (inMeta && !shouldBeDeployed && !isDeployed) {
711       return;
712     } else if (recentlyModified) {
713       LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
714       return;
715     }
716     // ========== Cases where the region is not in META =============
717     else if (!inMeta && !inHdfs && !isDeployed) {
718       // We shouldn't have record of this region at all then!
719       assert false : "Entry for region with no data";
720     } else if (!inMeta && !inHdfs && isDeployed) {
721       errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region "
722           + descriptiveName + ", key=" + key + ", not on HDFS or in META but " +
723           "deployed on " + Joiner.on(", ").join(hbi.deployedOn));
724     } else if (!inMeta && inHdfs && !isDeployed) {
725       errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region "
726           + descriptiveName + " on HDFS, but not listed in META " +
727           "or deployed on any region server");
728     } else if (!inMeta && inHdfs && isDeployed) {
729       errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
730           + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn));
731 
732     // ========== Cases where the region is in META =============
733     } else if (inMeta && !inHdfs && !isDeployed) {
734       errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region "
735           + descriptiveName + " found in META, but not in HDFS "
736           + "or deployed on any region server.");
737     } else if (inMeta && !inHdfs && isDeployed) {
738       errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName
739           + " found in META, but not in HDFS, " +
740           "and deployed on " + Joiner.on(", ").join(hbi.deployedOn));
741     } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) {
742       errors.reportError(ERROR_CODE.NOT_DEPLOYED, "Region " + descriptiveName
743           + " not deployed on any region server.");
744       // If we are trying to fix the errors
745       if (shouldFix()) {
746         errors.print("Trying to fix unassigned region...");
747         setShouldRerun();
748         HBaseFsckRepair.fixUnassigned(this.admin, hbi.metaEntry);
749       }
750     } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) {
751       errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, "Region "
752           + descriptiveName + " should not be deployed according " +
753           "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn));
754     } else if (inMeta && inHdfs && isMultiplyDeployed) {
755       errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName
756           + " is listed in META on region server " + hbi.metaEntry.regionServer
757           + " but is multiply assigned to region servers " +
758           Joiner.on(", ").join(hbi.deployedOn));
759       // If we are trying to fix the errors
760       if (shouldFix()) {
761         errors.print("Trying to fix assignment error...");
762         setShouldRerun();
763         HBaseFsckRepair.fixDupeAssignment(this.admin, hbi.metaEntry, hbi.deployedOn);
764       }
765     } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
766       errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region "
767           + descriptiveName + " listed in META on region server " +
768           hbi.metaEntry.regionServer + " but found on region server " +
769           hbi.deployedOn.get(0));
770       // If we are trying to fix the errors
771       if (shouldFix()) {
772         errors.print("Trying to fix assignment error...");
773         setShouldRerun();
774         HBaseFsckRepair.fixDupeAssignment(this.admin, hbi.metaEntry, hbi.deployedOn);
775       }
776     } else {
777       errors.reportError(ERROR_CODE.UNKNOWN, "Region " + descriptiveName +
778           " is in an unforeseen state:" +
779           " inMeta=" + inMeta +
780           " inHdfs=" + inHdfs +
781           " isDeployed=" + isDeployed +
782           " isMultiplyDeployed=" + isMultiplyDeployed +
783           " deploymentMatchesMeta=" + deploymentMatchesMeta +
784           " shouldBeDeployed=" + shouldBeDeployed);
785     }
786   }
787 
788   /**
789    * Checks tables integrity. Goes over all regions and scans the tables.
790    * Collects all the pieces for each table and checks if there are missing,
791    * repeated or overlapping ones.
792    */
793   void checkIntegrity() {
794     for (HbckInfo hbi : regionInfo.values()) {
795       // Check only valid, working regions
796       if (hbi.metaEntry == null) continue;
797       if (hbi.metaEntry.regionServer == null) continue;
798       if (hbi.onlyEdits) continue;
799 
800       // Missing regionDir or over-deployment is checked elsewhere. Include
801       // these cases in modTInfo, so we can evaluate those regions as part of
802       // the region chain in META
803       //if (hbi.foundRegionDir == null) continue;
804       //if (hbi.deployedOn.size() != 1) continue;
805       if (hbi.deployedOn.size() == 0) continue;
806 
807       // We should be safe here
808       String tableName = hbi.metaEntry.getTableNameAsString();
809       TInfo modTInfo = tablesInfo.get(tableName);
810       if (modTInfo == null) {
811         modTInfo = new TInfo(tableName);
812       }
813       for (ServerName server : hbi.deployedOn) {
814         modTInfo.addServer(server);
815       }
816 
817       modTInfo.addRegionInfo(hbi);
818 
819       tablesInfo.put(tableName, modTInfo);
820     }
821 
822     for (TInfo tInfo : tablesInfo.values()) {
823       if (!tInfo.checkRegionChain()) {
824         errors.report("Found inconsistency in table " + tInfo.getName());
825       }
826     }
827   }
828 
829   /**
830    * Maintain information about a particular table.
831    */
832   private class TInfo {
833     String tableName;
834     TreeSet <ServerName> deployedOn;
835 
836     final List<HbckInfo> backwards = new ArrayList<HbckInfo>();
837     final RegionSplitCalculator<HbckInfo> sc = new RegionSplitCalculator<HbckInfo>(cmp);
838 
839     // key = start split, values = set of splits in problem group
840     final Multimap<byte[], HbckInfo> overlapGroups = 
841       TreeMultimap.create(RegionSplitCalculator.BYTES_COMPARATOR, cmp);
842 
843     TInfo(String name) {
844       this.tableName = name;
845       deployedOn = new TreeSet <ServerName>();
846     }
847 
848     public void addRegionInfo(HbckInfo hir) {
849       if (Bytes.equals(hir.getEndKey(), HConstants.EMPTY_END_ROW)) {
850         // end key is absolute end key, just add it.
851         sc.add(hir);
852         return;
853       }
854 
855       // if not the absolute end key, check for cycle 
856       if (Bytes.compareTo(hir.getStartKey(), hir.getEndKey()) > 0) {
857         errors.reportError(
858             ERROR_CODE.REGION_CYCLE,
859             String.format("The endkey for this region comes before the "
860                 + "startkey, startkey=%s, endkey=%s",
861                 Bytes.toStringBinary(hir.getStartKey()),
862                 Bytes.toStringBinary(hir.getEndKey())), this, hir);
863         backwards.add(hir);
864         return;
865       }
866 
867       // main case, add to split calculator
868       sc.add(hir);
869     }
870 
871     public void addServer(ServerName server) {
872       this.deployedOn.add(server);
873     }
874 
875     public String getName() {
876       return tableName;
877     }
878 
879     public int getNumRegions() {
880       return sc.getStarts().size() + backwards.size();
881     }
882 
883     /**
884      * Check the region chain (from META) of this table.  We are looking for
885      * holes, overlaps, and cycles.
886      * @return false if there are errors
887      */
888     public boolean checkRegionChain() {
889       int originalErrorsCount = errors.getErrorList().size();
890       Multimap<byte[], HbckInfo> regions = sc.calcCoverage();
891       SortedSet<byte[]> splits = sc.getSplits();
892 
893       byte[] prevKey = null;
894       byte[] problemKey = null;
895       for (byte[] key : splits) {
896         Collection<HbckInfo> ranges = regions.get(key);
897         if (prevKey == null && !Bytes.equals(key, HConstants.EMPTY_BYTE_ARRAY)) {
898           for (HbckInfo rng : ranges) {
899             // TODO offline fix region hole.
900 
901             errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
902                 "First region should start with an empty key.  You need to "
903                 + " create a new region and regioninfo in HDFS to plug the hole.",
904                 this, rng);
905           }
906         }
907 
908         // check for degenerate ranges
909         for (HbckInfo rng : ranges) {
910           // special endkey case converts '' to null
911           byte[] endKey = rng.getEndKey();
912           endKey = (endKey.length == 0) ? null : endKey;
913           if (Bytes.equals(rng.getStartKey(),endKey)) {
914             errors.reportError(ERROR_CODE.DEGENERATE_REGION,
915               "Region has the same start and end key.", this, rng);
916           }
917         }
918 
919         if (ranges.size() == 1) {
920           // this split key is ok -- no overlap, not a hole.
921           if (problemKey != null) {
922             LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
923           }
924           problemKey = null; // fell through, no more problem.
925         } else if (ranges.size() > 1) {
926           // set the new problem key group name, if already have problem key, just
927           // keep using it.
928           if (problemKey == null) {
929             // only for overlap regions.
930             LOG.warn("Naming new problem group: " + Bytes.toStringBinary(key));
931             problemKey = key;
932           }
933           overlapGroups.putAll(problemKey, ranges);
934 
935           // record errors
936           ArrayList<HbckInfo> subRange = new ArrayList<HbckInfo>(ranges);
937           //  this dumb and n^2 but this shouldn't happen often
938           for (HbckInfo r1 : ranges) {
939             subRange.remove(r1);
940             for (HbckInfo r2 : subRange) {
941               if (Bytes.compareTo(r1.getStartKey(), r2.getStartKey())==0) {
942                 // dup start key
943                 errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
944                     "Multiple regions have the same startkey: "
945                     + Bytes.toStringBinary(key), this, r1);
946                 errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
947                     "Multiple regions have the same startkey: "
948                     + Bytes.toStringBinary(key), this, r2);
949               } else {
950                 // overlap
951                 errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
952                     "There is an overlap in the region chain.",
953                     this, r1);
954               }
955             }
956           }
957 
958         } else if (ranges.size() == 0) {
959           if (problemKey != null) {
960             LOG.warn("reached end of problem group: " + Bytes.toStringBinary(key));
961           }
962           problemKey = null;
963 
964           byte[] holeStopKey = sc.getSplits().higher(key);
965           // if higher key is null we reached the top.
966           if (holeStopKey != null) {
967             // hole
968             errors.reportError(ERROR_CODE.HOLE_IN_REGION_CHAIN,
969                 "There is a hole in the region chain between "
970                 + Bytes.toStringBinary(key) + " and "
971                 + Bytes.toStringBinary(holeStopKey)
972                 + ".  You need to create a new regioninfo and region "
973                 + "dir in hdfs to plug the hole.");
974           }
975         }
976         prevKey = key;
977       }
978 
979       if (details) {
980         // do full region split map dump
981         System.out.println("---- Table '"  +  this.tableName 
982             + "': region split map");
983         dump(splits, regions);
984         System.out.println("---- Table '"  +  this.tableName 
985             + "': overlap groups");
986         dumpOverlapProblems(overlapGroups);
987         System.out.println("There are " + overlapGroups.keySet().size()
988             + " overlap groups with " + overlapGroups.size()
989             + " overlapping regions");
990       }
991       return errors.getErrorList().size() == originalErrorsCount;
992     }
993 
994     /**
995      * This dumps data in a visually reasonable way for visual debugging
996      * 
997      * @param splits
998      * @param regions
999      */
1000     void dump(SortedSet<byte[]> splits, Multimap<byte[], HbckInfo> regions) {
1001       // we display this way because the last end key should be displayed as well.
1002       for (byte[] k : splits) {
1003         System.out.print(Bytes.toStringBinary(k) + ":\t");
1004         for (HbckInfo r : regions.get(k)) {
1005           System.out.print("[ "+ r.toString() + ", "
1006               + Bytes.toStringBinary(r.getEndKey())+ "]\t");
1007         }
1008         System.out.println();
1009       }
1010     }
1011   }
1012 
1013   public void dumpOverlapProblems(Multimap<byte[], HbckInfo> regions) {
1014     // we display this way because the last end key should be displayed as
1015     // well.
1016     for (byte[] k : regions.keySet()) {
1017       System.out.print(Bytes.toStringBinary(k) + ":\n");
1018       for (HbckInfo r : regions.get(k)) {
1019         System.out.print("[ " + r.toString() + ", "
1020             + Bytes.toStringBinary(r.getEndKey()) + "]\n");
1021       }
1022       System.out.println("----");
1023     }
1024   }
1025 
1026   public Multimap<byte[], HbckInfo> getOverlapGroups(String table) {
1027     return tablesInfo.get(table).overlapGroups;
1028   }
1029 
1030   /**
1031    * Return a list of user-space table names whose metadata have not been
1032    * modified in the last few milliseconds specified by timelag
1033    * if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER, STARTCODE_QUALIFIER,
1034    * SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last
1035    * milliseconds specified by timelag, then the table is a candidate to be returned.
1036    * @return tables that have not been modified recently
1037    * @throws IOException if an error is encountered
1038    */
1039    HTableDescriptor[] getTables(AtomicInteger numSkipped) {
1040     List<String> tableNames = new ArrayList<String>();
1041     long now = System.currentTimeMillis();
1042 
1043     for (HbckInfo hbi : regionInfo.values()) {
1044       MetaEntry info = hbi.metaEntry;
1045 
1046       // if the start key is zero, then we have found the first region of a table.
1047       // pick only those tables that were not modified in the last few milliseconds.
1048       if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
1049         if (info.modTime + timelag < now) {
1050           tableNames.add(info.getTableNameAsString());
1051         } else {
1052           numSkipped.incrementAndGet(); // one more in-flux table
1053         }
1054       }
1055     }
1056     return getHTableDescriptors(tableNames);
1057   }
1058 
1059    HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
1060     HTableDescriptor[] htd = null;
1061      try {
1062        LOG.info("getHTableDescriptors == tableNames => " + tableNames);
1063        htd = new HBaseAdmin(conf).getTableDescriptors(tableNames);
1064      } catch (IOException e) {
1065        LOG.debug("Exception getting table descriptors", e);
1066      }
1067      return htd;
1068   }
1069 
1070 
1071   /**
1072    * Gets the entry in regionInfo corresponding to the the given encoded
1073    * region name. If the region has not been seen yet, a new entry is added
1074    * and returned.
1075    */
1076   private synchronized HbckInfo getOrCreateInfo(String name) {
1077     HbckInfo hbi = regionInfo.get(name);
1078     if (hbi == null) {
1079       hbi = new HbckInfo(null);
1080       regionInfo.put(name, hbi);
1081     }
1082     return hbi;
1083   }
1084 
1085   /**
1086     * Check values in regionInfo for .META.
1087     * Check if zero or more than one regions with META are found.
1088     * If there are inconsistencies (i.e. zero or more than one regions
1089     * pretend to be holding the .META.) try to fix that and report an error.
1090     * @throws IOException from HBaseFsckRepair functions
1091    * @throws KeeperException
1092    * @throws InterruptedException
1093     */
1094   boolean checkMetaEntries()
1095   throws IOException, KeeperException, InterruptedException {
1096     List <HbckInfo> metaRegions = Lists.newArrayList();
1097     for (HbckInfo value : regionInfo.values()) {
1098       if (value.metaEntry.isMetaRegion()) {
1099         metaRegions.add(value);
1100       }
1101     }
1102 
1103     // If something is wrong
1104     if (metaRegions.size() != 1) {
1105       HRegionLocation rootLocation = connection.locateRegion(
1106         HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW);
1107       HbckInfo root =
1108           regionInfo.get(rootLocation.getRegionInfo().getEncodedName());
1109 
1110       // If there is no region holding .META.
1111       if (metaRegions.size() == 0) {
1112         errors.reportError(ERROR_CODE.NO_META_REGION, ".META. is not found on any region.");
1113         if (shouldFix()) {
1114           errors.print("Trying to fix a problem with .META...");
1115           setShouldRerun();
1116           // try to fix it (treat it as unassigned region)
1117           HBaseFsckRepair.fixUnassigned(this.admin, root.metaEntry);
1118         }
1119       }
1120       // If there are more than one regions pretending to hold the .META.
1121       else if (metaRegions.size() > 1) {
1122         errors.reportError(ERROR_CODE.MULTI_META_REGION, ".META. is found on more than one region.");
1123         if (shouldFix()) {
1124           errors.print("Trying to fix a problem with .META...");
1125           setShouldRerun();
1126           // try fix it (treat is a dupe assignment)
1127           List <ServerName> deployedOn = Lists.newArrayList();
1128           for (HbckInfo mRegion : metaRegions) {
1129             deployedOn.add(mRegion.metaEntry.regionServer);
1130           }
1131           HBaseFsckRepair.fixDupeAssignment(this.admin, root.metaEntry, deployedOn);
1132         }
1133       }
1134       // rerun hbck with hopefully fixed META
1135       return false;
1136     }
1137     // no errors, so continue normally
1138     return true;
1139   }
1140 
1141   /**
1142    * Scan .META. and -ROOT-, adding all regions found to the regionInfo map.
1143    * @throws IOException if an error is encountered
1144    */
1145   void getMetaEntries() throws IOException {
1146     MetaScannerVisitor visitor = new MetaScannerVisitor() {
1147       int countRecord = 1;
1148 
1149       // comparator to sort KeyValues with latest modtime
1150       final Comparator<KeyValue> comp = new Comparator<KeyValue>() {
1151         public int compare(KeyValue k1, KeyValue k2) {
1152           return (int)(k1.getTimestamp() - k2.getTimestamp());
1153         }
1154       };
1155 
1156       public boolean processRow(Result result) throws IOException {
1157         try {
1158 
1159           // record the latest modification of this META record
1160           long ts =  Collections.max(result.list(), comp).getTimestamp();
1161           Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(result);
1162           if (pair == null || pair.getFirst() == null) {
1163             emptyRegionInfoQualifiers.add(result);
1164             return true;
1165           }
1166           ServerName sn = null;
1167           if (pair.getSecond() != null) {
1168             sn = pair.getSecond();
1169           }
1170           MetaEntry m = new MetaEntry(pair.getFirst(), sn, ts);
1171           HbckInfo hbInfo = new HbckInfo(m);
1172           HbckInfo previous = regionInfo.put(pair.getFirst().getEncodedName(), hbInfo);
1173           if (previous != null) {
1174             throw new IOException("Two entries in META are same " + previous);
1175           }
1176 
1177           // show proof of progress to the user, once for every 100 records.
1178           if (countRecord % 100 == 0) {
1179             errors.progress();
1180           }
1181           countRecord++;
1182           return true;
1183         } catch (RuntimeException e) {
1184           LOG.error("Result=" + result);
1185           throw e;
1186         }
1187       }
1188     };
1189 
1190     // Scan -ROOT- to pick up META regions
1191     MetaScanner.metaScan(conf, visitor, null, null,
1192       Integer.MAX_VALUE, HConstants.ROOT_TABLE_NAME);
1193 
1194     if (!checkMetaOnly) {
1195       // Scan .META. to pick up user regions
1196       MetaScanner.metaScan(conf, visitor);
1197     }
1198     
1199     errors.print("");
1200   }
1201 
1202   /**
1203    * Stores the entries scanned from META
1204    */
1205   static class MetaEntry extends HRegionInfo {
1206     private static final Log LOG = LogFactory.getLog(HRegionInfo.class);
1207     ServerName regionServer;   // server hosting this region
1208     long modTime;          // timestamp of most recent modification metadata
1209 
1210     public MetaEntry(HRegionInfo rinfo, ServerName regionServer, long modTime) {
1211       super(rinfo);
1212       this.regionServer = regionServer;
1213       this.modTime = modTime;
1214     }
1215   }
1216 
1217   /**
1218    * Maintain information about a particular region.
1219    */
1220   public static class HbckInfo implements KeyRange {
1221     boolean onlyEdits = false;
1222     MetaEntry metaEntry = null;
1223     FileStatus foundRegionDir = null;
1224     List<ServerName> deployedOn = Lists.newArrayList();
1225     String hdfsTableName = null; // This is set in the workitem loader.
1226 
1227     HbckInfo(MetaEntry metaEntry) {
1228       this.metaEntry = metaEntry;
1229     }
1230 
1231     public synchronized void addServer(ServerName server) {
1232       this.deployedOn.add(server);
1233     }
1234 
1235     public synchronized String toString() {
1236       if (metaEntry != null) {
1237         return metaEntry.getRegionNameAsString();
1238       } else if (foundRegionDir != null) {
1239         return foundRegionDir.getPath().toString();
1240       } else {
1241         return "UNKNOWN_REGION on " + Joiner.on(", ").join(deployedOn);
1242       }
1243     }
1244 
1245     @Override
1246     public byte[] getStartKey() {
1247       return this.metaEntry.getStartKey();
1248     }
1249 
1250     @Override
1251     public byte[] getEndKey() {
1252       return this.metaEntry.getEndKey();
1253     }
1254   }
1255 
1256   final static Comparator<HbckInfo> cmp = new Comparator<HbckInfo>() {
1257     @Override
1258     public int compare(HbckInfo l, HbckInfo r) {
1259       if (l == r) {
1260         // same instance
1261         return 0;
1262       }
1263 
1264       int tableCompare = RegionSplitCalculator.BYTES_COMPARATOR.compare(
1265           l.metaEntry.getTableName(), r.metaEntry.getTableName());
1266       if (tableCompare != 0) {
1267         return tableCompare;
1268       }
1269 
1270       int startComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
1271           l.metaEntry.getStartKey(), r.metaEntry.getStartKey());
1272       if (startComparison != 0) {
1273         return startComparison;
1274       }
1275 
1276       // Special case for absolute endkey
1277       byte[] endKey = r.metaEntry.getEndKey();
1278       endKey = (endKey.length == 0) ? null : endKey;
1279       byte[] endKey2 = l.metaEntry.getEndKey();
1280       endKey2 = (endKey2.length == 0) ? null : endKey2;
1281       int endComparison = RegionSplitCalculator.BYTES_COMPARATOR.compare(
1282           endKey2,  endKey);
1283 
1284       if (endComparison != 0) {
1285         return endComparison;
1286       }
1287 
1288       // use modTime as tiebreaker.
1289       return (int) (l.metaEntry.modTime - r.metaEntry.modTime);
1290     }
1291   };
1292 
1293   /**
1294    * Prints summary of all tables found on the system.
1295    */
1296   private void printTableSummary() {
1297     System.out.println("Summary:");
1298     for (TInfo tInfo : tablesInfo.values()) {
1299       if (errors.tableHasErrors(tInfo)) {
1300         System.out.println("Table " + tInfo.getName() + " is inconsistent.");
1301       } else {
1302         System.out.println("  " + tInfo.getName() + " is okay.");
1303       }
1304       System.out.println("    Number of regions: " + tInfo.getNumRegions());
1305       System.out.print("    Deployed on: ");
1306       for (ServerName server : tInfo.deployedOn) {
1307         System.out.print(" " + server.toString());
1308       }
1309       System.out.println();
1310     }
1311   }
1312 
1313   public interface ErrorReporter {
1314     public static enum ERROR_CODE {
1315       UNKNOWN, NO_META_REGION, NULL_ROOT_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
1316       NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
1317       MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
1318       FIRST_REGION_STARTKEY_NOT_EMPTY, DUPE_STARTKEYS,
1319       HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE, DEGENERATE_REGION
1320     }
1321     public void clear();
1322     public void report(String message);
1323     public void reportError(String message);
1324     public void reportError(ERROR_CODE errorCode, String message);
1325     public void reportError(ERROR_CODE errorCode, String message, TInfo table, HbckInfo info);
1326     public void reportError(ERROR_CODE errorCode, String message, TInfo table, HbckInfo info1, HbckInfo info2);
1327     public int summarize();
1328     public void detail(String details);
1329     public ArrayList<ERROR_CODE> getErrorList();
1330     public void progress();
1331     public void print(String message);
1332     public void resetErrors();
1333     public boolean tableHasErrors(TInfo table);
1334   }
1335 
1336   private static class PrintingErrorReporter implements ErrorReporter {
1337     public int errorCount = 0;
1338     private int showProgress;
1339 
1340     Set<TInfo> errorTables = new HashSet<TInfo>();
1341 
1342     // for use by unit tests to verify which errors were discovered
1343     private ArrayList<ERROR_CODE> errorList = new ArrayList<ERROR_CODE>();
1344 
1345     public void clear() {
1346       errorTables.clear();
1347       errorList.clear();
1348       errorCount = 0;
1349     }
1350 
1351     public synchronized void reportError(ERROR_CODE errorCode, String message) {
1352       errorList.add(errorCode);
1353       if (!summary) {
1354         System.out.println("ERROR: " + message);
1355       }
1356       errorCount++;
1357       showProgress = 0;
1358     }
1359 
1360     public synchronized void reportError(ERROR_CODE errorCode, String message, TInfo table,
1361                                          HbckInfo info) {
1362       errorTables.add(table);
1363       String reference = "(region " + info.metaEntry.getRegionNameAsString() + ")";
1364       reportError(errorCode, reference + " " + message);
1365     }
1366 
1367     public synchronized void reportError(ERROR_CODE errorCode, String message, TInfo table,
1368                                          HbckInfo info1, HbckInfo info2) {
1369       errorTables.add(table);
1370       String reference = "(regions " + info1.metaEntry.getRegionNameAsString()
1371           + " and " + info2.metaEntry.getRegionNameAsString() + ")";
1372       reportError(errorCode, reference + " " + message);
1373     }
1374 
1375     public synchronized void reportError(String message) {
1376       reportError(ERROR_CODE.UNKNOWN, message);
1377     }
1378 
1379     /**
1380      * Report error information, but do not increment the error count.  Intended for cases
1381      * where the actual error would have been reported previously.
1382      * @param message
1383      */
1384     public synchronized void report(String message) {
1385       if (! summary) {
1386         System.out.println("ERROR: " + message);
1387       }
1388       showProgress = 0;
1389     }
1390 
1391     public synchronized int summarize() {
1392       System.out.println(Integer.toString(errorCount) +
1393                          " inconsistencies detected.");
1394       if (errorCount == 0) {
1395         System.out.println("Status: OK");
1396         return 0;
1397       } else {
1398         System.out.println("Status: INCONSISTENT");
1399         return -1;
1400       }
1401     }
1402 
1403     public ArrayList<ERROR_CODE> getErrorList() {
1404       return errorList;
1405     }
1406 
1407     public synchronized void print(String message) {
1408       if (!summary) {
1409         System.out.println(message);
1410       }
1411     }
1412 
1413     @Override
1414     public boolean tableHasErrors(TInfo table) {
1415       return errorTables.contains(table);
1416     }
1417 
1418     @Override
1419     public void resetErrors() {
1420       errorCount = 0;
1421     }
1422 
1423     public synchronized void detail(String message) {
1424       if (details) {
1425         System.out.println(message);
1426       }
1427       showProgress = 0;
1428     }
1429 
1430     public synchronized void progress() {
1431       if (showProgress++ == 10) {
1432         if (!summary) {
1433           System.out.print(".");
1434         }
1435         showProgress = 0;
1436       }
1437     }
1438   }
1439 
1440   /**
1441    * Contact a region server and get all information from it
1442    */
1443   static class WorkItemRegion implements Runnable {
1444     private HBaseFsck hbck;
1445     private ServerName rsinfo;
1446     private ErrorReporter errors;
1447     private HConnection connection;
1448     private boolean done;
1449 
1450     WorkItemRegion(HBaseFsck hbck, ServerName info,
1451                    ErrorReporter errors, HConnection connection) {
1452       this.hbck = hbck;
1453       this.rsinfo = info;
1454       this.errors = errors;
1455       this.connection = connection;
1456       this.done = false;
1457     }
1458 
1459     // is this task done?
1460     synchronized boolean isDone() {
1461       return done;
1462     }
1463 
1464     @Override
1465     public synchronized void run() {
1466       errors.progress();
1467       try {
1468         HRegionInterface server = connection.getHRegionConnection(new HServerAddress(rsinfo.getHostname(), rsinfo.getPort()));
1469 
1470         // list all online regions from this region server
1471         List<HRegionInfo> regions = server.getOnlineRegions();
1472         if (hbck.checkMetaOnly) {
1473           regions = filterOnlyMetaRegions(regions);
1474         }
1475         if (details) {
1476           errors.detail("RegionServer: " + rsinfo.getServerName() +
1477                            " number of regions: " + regions.size());
1478           for (HRegionInfo rinfo: regions) {
1479             errors.detail("  " + rinfo.getRegionNameAsString() +
1480                              " id: " + rinfo.getRegionId() +
1481                              " encoded_name: " + rinfo.getEncodedName() +
1482                              " start: " + Bytes.toStringBinary(rinfo.getStartKey()) +
1483                              " end: " + Bytes.toStringBinary(rinfo.getEndKey()));
1484           }
1485         }
1486 
1487         // check to see if the existence of this region matches the region in META
1488         for (HRegionInfo r:regions) {
1489           HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
1490           hbi.addServer(rsinfo);
1491         }
1492       } catch (IOException e) {          // unable to connect to the region server. 
1493         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
1494           " Unable to fetch region information. " + e);
1495       } finally {
1496         done = true;
1497         notifyAll(); // wakeup anybody waiting for this item to be done
1498       }
1499     }
1500 
1501     private List<HRegionInfo> filterOnlyMetaRegions(List<HRegionInfo> regions) {
1502       List<HRegionInfo> ret = Lists.newArrayList();
1503       for (HRegionInfo hri : regions) {
1504         if (hri.isMetaTable()) {
1505           ret.add(hri);
1506         }
1507       }
1508       return ret;
1509     }
1510   }
1511 
1512   /**
1513    * Contact hdfs and get all information about specified table directory.
1514    */
1515   static class WorkItemHdfsDir implements Runnable {
1516     private HBaseFsck hbck;
1517     private FileStatus tableDir;
1518     private ErrorReporter errors;
1519     private FileSystem fs;
1520     private boolean done;
1521 
1522     WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors, 
1523                     FileStatus status) {
1524       this.hbck = hbck;
1525       this.fs = fs;
1526       this.tableDir = status;
1527       this.errors = errors;
1528       this.done = false;
1529     }
1530 
1531     synchronized boolean isDone() {
1532       return done;
1533     } 
1534 
1535     @Override
1536     public synchronized void run() {
1537       try {
1538         String tableName = tableDir.getPath().getName();
1539         // ignore hidden files
1540         if (tableName.startsWith(".") &&
1541             !tableName.equals( Bytes.toString(HConstants.META_TABLE_NAME)))
1542           return;
1543         // level 2: <HBASE_DIR>/<table>/*
1544         FileStatus[] regionDirs = fs.listStatus(tableDir.getPath());
1545         for (FileStatus regionDir : regionDirs) {
1546           String encodedName = regionDir.getPath().getName();
1547 
1548           // ignore directories that aren't hexadecimal
1549           if (!encodedName.toLowerCase().matches("[0-9a-f]+")) continue;
1550   
1551           HbckInfo hbi = hbck.getOrCreateInfo(encodedName);
1552           hbi.hdfsTableName = tableName;
1553           synchronized (hbi) {
1554             if (hbi.foundRegionDir != null) {
1555               errors.print("Directory " + encodedName + " duplicate??" +
1556                            hbi.foundRegionDir);
1557             }
1558             hbi.foundRegionDir = regionDir;
1559         
1560             // Set a flag if this region contains only edits
1561             // This is special case if a region is left after split
1562             hbi.onlyEdits = true;
1563             FileStatus[] subDirs = fs.listStatus(regionDir.getPath());
1564             Path ePath = HLog.getRegionDirRecoveredEditsDir(regionDir.getPath());
1565             for (FileStatus subDir : subDirs) {
1566               String sdName = subDir.getPath().getName();
1567               if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) {
1568                 hbi.onlyEdits = false;
1569                 break;
1570               }
1571             }
1572           }
1573         }
1574       } catch (IOException e) {          // unable to connect to the region server. 
1575         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "Table Directory: " + tableDir.getPath().getName() +
1576                       " Unable to fetch region information. " + e);
1577       } finally {
1578         done = true;
1579         notifyAll();
1580       }
1581     }
1582   }
1583 
1584   /**
1585    * Display the full report from fsck. This displays all live and dead region
1586    * servers, and all known regions.
1587    */
1588   public void displayFullReport() {
1589     details = true;
1590   }
1591 
1592   /**
1593    * Set summary mode.
1594    * Print only summary of the tables and status (OK or INCONSISTENT)
1595    */
1596   void setSummary() {
1597     summary = true;
1598   }
1599 
1600   /**
1601    * Set META check mode.
1602    * Print only info about META table deployment/state
1603    */
1604   void setCheckMetaOnly() {
1605     checkMetaOnly = true;
1606   }
1607 
1608   /**
1609    * Check if we should rerun fsck again. This checks if we've tried to
1610    * fix something and we should rerun fsck tool again.
1611    * Display the full report from fsck. This displays all live and dead
1612    * region servers, and all known regions.
1613    */
1614   void setShouldRerun() {
1615     rerun = true;
1616   }
1617 
1618   boolean shouldRerun() {
1619     return rerun;
1620   }
1621 
1622   /**
1623    * Fix inconsistencies found by fsck. This should try to fix errors (if any)
1624    * found by fsck utility.
1625    */
1626   public void setFixErrors(boolean shouldFix) {
1627     fix = shouldFix;
1628   }
1629 
1630   boolean shouldFix() {
1631     return fix;
1632   }
1633 
1634   /**
1635    * We are interested in only those tables that have not changed their state in
1636    * META during the last few seconds specified by hbase.admin.fsck.timelag
1637    * @param seconds - the time in seconds
1638    */
1639   public void setTimeLag(long seconds) {
1640     timelag = seconds * 1000; // convert to milliseconds
1641   }
1642 
1643   protected static void printUsageAndExit() {
1644     System.err.println("Usage: fsck [opts] ");
1645     System.err.println(" where [opts] are:");
1646     System.err.println("   -details Display full report of all regions.");
1647     System.err.println("   -timelag {timeInSeconds}  Process only regions that " +
1648                        " have not experienced any metadata updates in the last " +
1649                        " {{timeInSeconds} seconds.");
1650     System.err.println("   -fix Try to fix some of the errors.");
1651     System.err.println("   -sleepBeforeRerun {timeInSeconds} Sleep this many seconds" +
1652                        " before checking if the fix worked if run with -fix");
1653     System.err.println("   -summary Print only summary of the tables and status.");
1654     System.err.println("   -metaonly Only check the state of ROOT and META tables.");
1655     Runtime.getRuntime().exit(-2);
1656   }
1657 
1658   /**
1659    * Main program
1660    * @param args
1661    * @throws Exception
1662    */
1663   public static void main(String[] args) throws Exception {
1664 
1665     // create a fsck object
1666     Configuration conf = HBaseConfiguration.create();
1667     conf.set("fs.defaultFS", conf.get(HConstants.HBASE_DIR));
1668     HBaseFsck fsck = new HBaseFsck(conf);
1669     long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
1670 
1671     // Process command-line args.
1672     for (int i = 0; i < args.length; i++) {
1673       String cmd = args[i];
1674       if (cmd.equals("-details")) {
1675         fsck.displayFullReport();
1676       } else if (cmd.equals("-timelag")) {
1677         if (i == args.length - 1) {
1678           System.err.println("HBaseFsck: -timelag needs a value.");
1679           printUsageAndExit();
1680         }
1681         try {
1682           long timelag = Long.parseLong(args[i+1]);
1683           fsck.setTimeLag(timelag);
1684         } catch (NumberFormatException e) {
1685           System.err.println("-timelag needs a numeric value.");
1686           printUsageAndExit();
1687         }
1688         i++;
1689       } else if (cmd.equals("-sleepBeforeRerun")) {
1690         if (i == args.length - 1) {
1691           System.err.println("HBaseFsck: -sleepBeforeRerun needs a value.");
1692           printUsageAndExit();
1693         }
1694         try {
1695           sleepBeforeRerun = Long.parseLong(args[i+1]);
1696         } catch (NumberFormatException e) {
1697           System.err.println("-sleepBeforeRerun needs a numeric value.");
1698           printUsageAndExit();
1699         }
1700         i++;
1701       } else if (cmd.equals("-fix")) {
1702         fsck.setFixErrors(true);
1703       } else if (cmd.equals("-summary")) {
1704         fsck.setSummary();
1705       } else if (cmd.equals("-metaonly")) {
1706         fsck.setCheckMetaOnly();
1707       } else {
1708         String str = "Unknown command line option : " + cmd;
1709         LOG.info(str);
1710         System.out.println(str);
1711         printUsageAndExit();
1712       }
1713     }
1714     // do the real work of fsck
1715     fsck.connect();
1716     int code = fsck.doWork();
1717     // If we have changed the HBase state it is better to run fsck again
1718     // to see if we haven't broken something else in the process.
1719     // We run it only once more because otherwise we can easily fall into
1720     // an infinite loop.
1721     if (fsck.shouldRerun()) {
1722       try {
1723         LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix...");
1724         Thread.sleep(sleepBeforeRerun);
1725       } catch (InterruptedException ie) {
1726         Runtime.getRuntime().exit(code);
1727       }
1728       // Just report
1729       fsck.setFixErrors(false);
1730       fsck.errors.resetErrors();
1731       code = fsck.doWork();
1732     }
1733 
1734     Runtime.getRuntime().exit(code);
1735   }
1736 }