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.*;
24  import java.util.concurrent.atomic.AtomicInteger;
25  import java.util.concurrent.ThreadPoolExecutor;
26  import java.util.concurrent.TimeUnit;
27  import java.util.concurrent.LinkedBlockingQueue;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.FileStatus;
33  import org.apache.hadoop.fs.FileSystem;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.ClusterStatus;
36  import org.apache.hadoop.hbase.HBaseConfiguration;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.HRegionLocation;
40  import org.apache.hadoop.hbase.HServerAddress;
41  import org.apache.hadoop.hbase.HServerInfo;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.MasterNotRunningException;
45  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
46  import org.apache.hadoop.hbase.client.HBaseAdmin;
47  import org.apache.hadoop.hbase.client.HConnection;
48  import org.apache.hadoop.hbase.client.HConnectionManager;
49  import org.apache.hadoop.hbase.client.MetaScanner;
50  import org.apache.hadoop.hbase.client.Result;
51  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
52  import org.apache.hadoop.hbase.ipc.HRegionInterface;
53  import org.apache.hadoop.hbase.regionserver.wal.HLog;
54  import org.apache.hadoop.hbase.zookeeper.ZKTable;
55  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
56  import org.apache.zookeeper.KeeperException;
57  
58  import com.google.common.base.Joiner;
59  import com.google.common.collect.Lists;
60  
61  import static org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
62  
63  /**
64   * Check consistency among the in-memory states of the master and the
65   * region server(s) and the state of data in HDFS.
66   */
67  public class HBaseFsck {
68    public static final long DEFAULT_TIME_LAG = 60000; // default value of 1 minute
69    public static final long DEFAULT_SLEEP_BEFORE_RERUN = 10000;
70  
71    private static final int MAX_NUM_THREADS = 50; // #threads to contact regions
72    private static final long THREADS_KEEP_ALIVE_SECONDS = 60;
73  
74    private static final Log LOG = LogFactory.getLog(HBaseFsck.class.getName());
75    private Configuration conf;
76  
77    private ClusterStatus status;
78    private HConnection connection;
79  
80    private TreeMap<String, HbckInfo> regionInfo = new TreeMap<String, HbckInfo>();
81    private TreeMap<String, TInfo> tablesInfo = new TreeMap<String, TInfo>();
82    private TreeSet<byte[]> disabledTables =
83      new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
84    ErrorReporter errors = new PrintingErrorReporter();
85  
86    private static boolean details = false; // do we display the full report
87    private long timelag = DEFAULT_TIME_LAG; // tables whose modtime is older
88    private boolean fix = false; // do we want to try fixing the errors?
89    private boolean rerun = false; // if we tried to fix something rerun hbck
90    private static boolean summary = false; // if we want to print less output
91    // Empty regioninfo qualifiers in .META.
92    private Set<Result> emptyRegionInfoQualifiers = new HashSet<Result>();
93    private int numThreads = MAX_NUM_THREADS;
94  
95    ThreadPoolExecutor executor; // threads to retrieve data from regionservers
96  
97    /**
98     * Constructor
99     *
100    * @param conf Configuration object
101    * @throws MasterNotRunningException if the master is not running
102    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
103    */
104   public HBaseFsck(Configuration conf)
105     throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
106     this.conf = conf;
107 
108     HBaseAdmin admin = new HBaseAdmin(conf);
109     status = admin.getMaster().getClusterStatus();
110     connection = admin.getConnection();
111 
112     numThreads = conf.getInt("hbasefsck.numthreads", numThreads);
113     executor = new ThreadPoolExecutor(0, numThreads,
114           THREADS_KEEP_ALIVE_SECONDS, TimeUnit.SECONDS,
115           new LinkedBlockingQueue<Runnable>());
116   }
117 
118   /**
119    * Contacts the master and prints out cluster-wide information
120    * @throws IOException if a remote or network exception occurs
121    * @return 0 on success, non-zero on failure
122    * @throws KeeperException
123    * @throws InterruptedException
124    */
125   int doWork() throws IOException, KeeperException, InterruptedException {
126     // print hbase server version
127     errors.print("Version: " + status.getHBaseVersion());
128 
129     // Make sure regionInfo is empty before starting
130     regionInfo.clear();
131     tablesInfo.clear();
132     emptyRegionInfoQualifiers.clear();
133     disabledTables.clear();
134     errors.clear();
135 
136     // get a list of all regions from the master. This involves
137     // scanning the META table
138     if (!recordRootRegion()) {
139       // Will remove later if we can fix it
140       errors.reportError("Encountered fatal error. Exiting...");
141       return -1;
142     }
143     getMetaEntries();
144 
145     // Check if .META. is found only once and in the right place
146     if (!checkMetaEntries()) {
147       // Will remove later if we can fix it
148       errors.reportError("Encountered fatal error. Exiting...");
149       return -1;
150     }
151 
152     // get a list of all tables that have not changed recently.
153     AtomicInteger numSkipped = new AtomicInteger(0);
154     HTableDescriptor[] allTables = getTables(numSkipped);
155     errors.print("Number of Tables: " + allTables.length);
156     if (details) {
157       if (numSkipped.get() > 0) {
158         errors.detail("Number of Tables in flux: " + numSkipped.get());
159       }
160       for (HTableDescriptor td : allTables) {
161         String tableName = td.getNameAsString();
162         errors.detail("  Table: " + tableName + "\t" +
163                            (td.isReadOnly() ? "ro" : "rw") + "\t" +
164                            (td.isRootRegion() ? "ROOT" :
165                             (td.isMetaRegion() ? "META" : "    ")) + "\t" +
166                            " families: " + td.getFamilies().size());
167       }
168     }
169 
170     // From the master, get a list of all known live region servers
171     Collection<HServerInfo> regionServers = status.getServerInfo();
172     errors.print("Number of live region servers: " +
173                        regionServers.size());
174     if (details) {
175       for (HServerInfo rsinfo: regionServers) {
176         errors.print("  " + rsinfo.getServerName());
177       }
178     }
179 
180     // From the master, get a list of all dead region servers
181     Collection<String> deadRegionServers = status.getDeadServerNames();
182     errors.print("Number of dead region servers: " +
183                        deadRegionServers.size());
184     if (details) {
185       for (String name: deadRegionServers) {
186         errors.print("  " + name);
187       }
188     }
189 
190     // Determine what's deployed
191     processRegionServers(regionServers);
192 
193     // Determine what's on HDFS
194     checkHdfs();
195 
196     // Empty cells in .META.?
197     errors.print("Number of empty REGIONINFO_QUALIFIER rows in .META.: " +
198       emptyRegionInfoQualifiers.size());
199     if (details) {
200       for (Result r: emptyRegionInfoQualifiers) {
201         errors.print("  " + r);
202       }
203     }
204 
205     // Get disabled tables from ZooKeeper
206     loadDisabledTables();
207 
208     // Check consistency
209     checkConsistency();
210 
211     // Check integrity
212     checkIntegrity();
213 
214     // Print table summary
215     printTableSummary();
216 
217     return errors.summarize();
218   }
219 
220   public ErrorReporter getErrors() {
221     return errors;
222   }
223 
224   /**
225    * Load the list of disabled tables in ZK into local set.
226    * @throws ZooKeeperConnectionException
227    * @throws IOException
228    * @throws KeeperException
229    */
230   private void loadDisabledTables()
231   throws ZooKeeperConnectionException, IOException, KeeperException {
232     ZooKeeperWatcher zkw =
233       HConnectionManager.getConnection(conf).getZooKeeperWatcher();
234     for (String tableName : ZKTable.getDisabledOrDisablingTables(zkw)) {
235       disabledTables.add(Bytes.toBytes(tableName));
236     }
237   }
238 
239   /**
240    * Check if the specified region's table is disabled.
241    * @throws ZooKeeperConnectionException
242    * @throws IOException
243    * @throws KeeperException
244    */
245   private boolean isTableDisabled(HRegionInfo regionInfo) {
246     return disabledTables.contains(regionInfo.getTableDesc().getName());
247   }
248 
249   /**
250    * Scan HDFS for all regions, recording their information into
251    * regionInfo
252    */
253   void checkHdfs() throws IOException, InterruptedException {
254     Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
255     FileSystem fs = rootDir.getFileSystem(conf);
256 
257     // list all tables from HDFS
258     List<FileStatus> tableDirs = Lists.newArrayList();
259 
260     boolean foundVersionFile = false;
261     FileStatus[] files = fs.listStatus(rootDir);
262     for (FileStatus file : files) {
263       if (file.getPath().getName().equals(HConstants.VERSION_FILE_NAME)) {
264         foundVersionFile = true;
265       } else {
266         tableDirs.add(file);
267       }
268     }
269 
270     // verify that version file exists
271     if (!foundVersionFile) {
272       errors.reportError(ERROR_CODE.NO_VERSION_FILE,
273           "Version file does not exist in root dir " + rootDir);
274     }
275 
276     // level 1:  <HBASE_DIR>/*
277     WorkItemHdfsDir[] dirs = new WorkItemHdfsDir[tableDirs.size()];  
278     int num = 0;
279     for (FileStatus tableDir : tableDirs) {
280       dirs[num] = new WorkItemHdfsDir(this, fs, errors, tableDir); 
281       executor.execute(dirs[num]);
282       num++;
283     }
284 
285     // wait for all directories to be done
286     for (int i = 0; i < num; i++) {
287       synchronized (dirs[i]) {
288         while (!dirs[i].isDone()) {
289           dirs[i].wait();
290         }
291       }
292     }
293   }
294 
295   /**
296    * Record the location of the ROOT region as found in ZooKeeper,
297    * as if it were in a META table. This is so that we can check
298    * deployment of ROOT.
299    */
300   boolean recordRootRegion() throws IOException {
301     HRegionLocation rootLocation = connection.locateRegion(
302       HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW);
303 
304     // Check if Root region is valid and existing
305     if (rootLocation == null || rootLocation.getRegionInfo() == null ||
306         rootLocation.getServerAddress() == null) {
307       errors.reportError(ERROR_CODE.NULL_ROOT_REGION,
308           "Root Region or some of its attributes are null.");
309       return false;
310     }
311 
312     MetaEntry m = new MetaEntry(rootLocation.getRegionInfo(),
313       rootLocation.getServerAddress(), null, System.currentTimeMillis());
314     HbckInfo hbInfo = new HbckInfo(m);
315     regionInfo.put(rootLocation.getRegionInfo().getEncodedName(), hbInfo);
316     return true;
317   }
318 
319   /**
320    * Contacts each regionserver and fetches metadata about regions.
321    * @param regionServerList - the list of region servers to connect to
322    * @throws IOException if a remote or network exception occurs
323    */
324   void processRegionServers(Collection<HServerInfo> regionServerList)
325     throws IOException, InterruptedException {
326 
327     WorkItemRegion[] work = new WorkItemRegion[regionServerList.size()];
328     int num = 0;
329 
330     // loop to contact each region server in parallel
331     for (HServerInfo rsinfo:regionServerList) {
332       work[num] = new WorkItemRegion(this, rsinfo, errors, connection);
333       executor.execute(work[num]);
334       num++;
335     }
336     
337     // wait for all submitted tasks to be done
338     for (int i = 0; i < num; i++) {
339       synchronized (work[i]) {
340         while (!work[i].isDone()) {
341           work[i].wait();
342         }
343       }
344     }
345   }
346 
347   /**
348    * Check consistency of all regions that have been found in previous phases.
349    * @throws KeeperException
350    * @throws InterruptedException
351    */
352   void checkConsistency()
353   throws IOException, KeeperException, InterruptedException {
354     for (java.util.Map.Entry<String, HbckInfo> e: regionInfo.entrySet()) {
355       doConsistencyCheck(e.getKey(), e.getValue());
356     }
357   }
358 
359   /**
360    * Check a single region for consistency and correct deployment.
361    * @throws KeeperException
362    * @throws InterruptedException
363    */
364   void doConsistencyCheck(final String key, final HbckInfo hbi)
365   throws IOException, KeeperException, InterruptedException {
366     String descriptiveName = hbi.toString();
367 
368     boolean inMeta = hbi.metaEntry != null;
369     boolean inHdfs = hbi.foundRegionDir != null;
370     boolean hasMetaAssignment = inMeta && hbi.metaEntry.regionServer != null;
371     boolean isDeployed = !hbi.deployedOn.isEmpty();
372     boolean isMultiplyDeployed = hbi.deployedOn.size() > 1;
373     boolean deploymentMatchesMeta =
374       hasMetaAssignment && isDeployed && !isMultiplyDeployed &&
375       hbi.metaEntry.regionServer.equals(hbi.deployedOn.get(0));
376     boolean splitParent =
377       (hbi.metaEntry == null)? false: hbi.metaEntry.isSplit() && hbi.metaEntry.isOffline();
378     boolean shouldBeDeployed = inMeta && !isTableDisabled(hbi.metaEntry);
379     boolean recentlyModified = hbi.foundRegionDir != null &&
380       hbi.foundRegionDir.getModificationTime() + timelag > System.currentTimeMillis();
381 
382     // ========== First the healthy cases =============
383     if (hbi.onlyEdits) {
384       return;
385     }
386     if (inMeta && inHdfs && isDeployed && deploymentMatchesMeta && shouldBeDeployed) {
387       return;
388     } else if (inMeta && !isDeployed && splitParent) {
389       // Offline regions shouldn't cause complaints
390       LOG.debug("Region " + descriptiveName + " offline, split, parent, ignoring.");
391       return;
392     } else if (inMeta && !shouldBeDeployed && !isDeployed) {
393       // offline regions shouldn't cause complaints
394       LOG.debug("Region " + descriptiveName + " offline, ignoring.");
395       return;
396     } else if (recentlyModified) {
397       LOG.warn("Region " + descriptiveName + " was recently modified -- skipping");
398       return;
399     }
400     // ========== Cases where the region is not in META =============
401     else if (!inMeta && !inHdfs && !isDeployed) {
402       // We shouldn't have record of this region at all then!
403       assert false : "Entry for region with no data";
404     } else if (!inMeta && !inHdfs && isDeployed) {
405       errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region "
406           + descriptiveName + ", key=" + key + ", not on HDFS or in META but " +
407           "deployed on " + Joiner.on(", ").join(hbi.deployedOn));
408     } else if (!inMeta && inHdfs && !isDeployed) {
409       errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region "
410           + descriptiveName + " on HDFS, but not listed in META " +
411           "or deployed on any region server.");
412     } else if (!inMeta && inHdfs && isDeployed) {
413       errors.reportError(ERROR_CODE.NOT_IN_META, "Region " + descriptiveName
414           + " not in META, but deployed on " + Joiner.on(", ").join(hbi.deployedOn));
415 
416     // ========== Cases where the region is in META =============
417     } else if (inMeta && !inHdfs && !isDeployed) {
418       errors.reportError(ERROR_CODE.NOT_IN_HDFS_OR_DEPLOYED, "Region "
419           + descriptiveName + " found in META, but not in HDFS "
420           + "or deployed on any region server.");
421     } else if (inMeta && !inHdfs && isDeployed) {
422       errors.reportError(ERROR_CODE.NOT_IN_HDFS, "Region " + descriptiveName
423           + " found in META, but not in HDFS, " +
424           "and deployed on " + Joiner.on(", ").join(hbi.deployedOn));
425     } else if (inMeta && inHdfs && !isDeployed && shouldBeDeployed) {
426       errors.reportError(ERROR_CODE.NOT_DEPLOYED, "Region " + descriptiveName
427           + " not deployed on any region server.");
428       // If we are trying to fix the errors
429       if (shouldFix()) {
430         errors.print("Trying to fix unassigned region...");
431         setShouldRerun();
432         HBaseFsckRepair.fixUnassigned(this.conf, hbi.metaEntry);
433       }
434     } else if (inMeta && inHdfs && isDeployed && !shouldBeDeployed) {
435       errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, "Region "
436           + descriptiveName + " should not be deployed according " +
437           "to META, but is deployed on " + Joiner.on(", ").join(hbi.deployedOn));
438     } else if (inMeta && inHdfs && isMultiplyDeployed) {
439       errors.reportError(ERROR_CODE.MULTI_DEPLOYED, "Region " + descriptiveName
440           + " is listed in META on region server " + hbi.metaEntry.regionServer
441           + " but is multiply assigned to region servers " +
442           Joiner.on(", ").join(hbi.deployedOn));
443       // If we are trying to fix the errors
444       if (shouldFix()) {
445         errors.print("Trying to fix assignment error...");
446         setShouldRerun();
447         HBaseFsckRepair.fixDupeAssignment(this.conf, hbi.metaEntry, hbi.deployedOn);
448       }
449     } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) {
450       errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, "Region "
451           + descriptiveName + " listed in META on region server " +
452           hbi.metaEntry.regionServer + " but found on region server " +
453           hbi.deployedOn.get(0));
454       // If we are trying to fix the errors
455       if (shouldFix()) {
456         errors.print("Trying to fix assignment error...");
457         setShouldRerun();
458         HBaseFsckRepair.fixDupeAssignment(this.conf, hbi.metaEntry, hbi.deployedOn);
459       }
460     } else {
461       errors.reportError(ERROR_CODE.UNKNOWN, "Region " + descriptiveName +
462           " is in an unforeseen state:" +
463           " inMeta=" + inMeta +
464           " inHdfs=" + inHdfs +
465           " isDeployed=" + isDeployed +
466           " isMultiplyDeployed=" + isMultiplyDeployed +
467           " deploymentMatchesMeta=" + deploymentMatchesMeta +
468           " shouldBeDeployed=" + shouldBeDeployed);
469     }
470   }
471 
472   /**
473    * Checks tables integrity. Goes over all regions and scans the tables.
474    * Collects all the pieces for each table and checks if there are missing,
475    * repeated or overlapping ones.
476    */
477   void checkIntegrity() {
478     for (HbckInfo hbi : regionInfo.values()) {
479       // Check only valid, working regions
480       if (hbi.metaEntry == null) continue;
481       if (hbi.metaEntry.regionServer == null) continue;
482       if (hbi.onlyEdits) continue;
483 
484       // Missing regionDir or over-deployment is checked elsewhere. Include
485       // these cases in modTInfo, so we can evaluate those regions as part of
486       // the region chain in META
487       //if (hbi.foundRegionDir == null) continue;
488       //if (hbi.deployedOn.size() != 1) continue;
489       if (hbi.deployedOn.size() == 0) continue;
490 
491       // We should be safe here
492       String tableName = hbi.metaEntry.getTableDesc().getNameAsString();
493       TInfo modTInfo = tablesInfo.get(tableName);
494       if (modTInfo == null) {
495         modTInfo = new TInfo(tableName);
496       }
497       for (HServerAddress server : hbi.deployedOn) {
498         modTInfo.addServer(server);
499       }
500 
501       //modTInfo.addEdge(hbi.metaEntry.getStartKey(), hbi.metaEntry.getEndKey());
502       modTInfo.addRegionInfo(hbi);
503 
504       tablesInfo.put(tableName, modTInfo);
505     }
506 
507     for (TInfo tInfo : tablesInfo.values()) {
508       if (!tInfo.checkRegionChain()) {
509         errors.report("Found inconsistency in table " + tInfo.getName());
510       }
511     }
512   }
513 
514   /**
515    * Maintain information about a particular table.
516    */
517   private class TInfo {
518     String tableName;
519     TreeSet <HServerAddress> deployedOn;
520 
521     List<HbckInfo> regions = new ArrayList<HbckInfo>();
522 
523     TInfo(String name) {
524       this.tableName = name;
525       deployedOn = new TreeSet <HServerAddress>();
526     }
527 
528     public void addRegionInfo (HbckInfo r) {
529       regions.add(r);
530     }
531 
532     public void addServer(HServerAddress server) {
533       this.deployedOn.add(server);
534     }
535 
536     public String getName() {
537       return tableName;
538     }
539 
540     public int getNumRegions() {
541       return regions.size();
542     }
543 
544     /**
545      * Check the region chain (from META) of this table.  We are looking for
546      * holes, overlaps, and cycles.
547      * @return false if there are errors
548      */
549     public boolean checkRegionChain() {
550       Collections.sort(regions);
551       HbckInfo last = null;
552 
553       for (HbckInfo r : regions) {
554         if (last == null) {
555           // This is the first region, check that the start key is empty
556           if (! Bytes.equals(r.metaEntry.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
557             errors.reportError(ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY,
558                 "First region should start with an empty key.",
559                 this, r);
560           }
561         } else {
562 
563           // Check if endKey < startKey
564           // Previous implementation of this code checked for a cycle in the
565           // region chain.  A cycle would imply that the endKey comes before
566           // the startKey (i.e. endKey < startKey).
567           if (! Bytes.equals(r.metaEntry.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)) {
568             // continue with this check if this is not the last region
569             int cmpRegionKeys = Bytes.compareTo(r.metaEntry.getStartKey(),
570                 r.metaEntry.getEndKey());
571             if (cmpRegionKeys > 0) {
572               errors.reportError(ERROR_CODE.REGION_CYCLE,
573                   String.format("The endkey for this region comes before the "
574                       + "startkey, startkey=%s, endkey=%s",
575                       Bytes.toString(r.metaEntry.getStartKey()),
576                       Bytes.toString(r.metaEntry.getEndKey())),
577                   this, r, last);
578             }
579           }
580 
581           // Check if the startkeys are different
582           if (Bytes.equals(r.metaEntry.getStartKey(), last.metaEntry.getStartKey())) {
583             errors.reportError(ERROR_CODE.DUPE_STARTKEYS,
584                 "Two regions have the same startkey: "
585                     + Bytes.toString(r.metaEntry.getStartKey()),
586                 this, r, last);
587           } else {
588             // Check that the startkey is the same as the previous end key
589             int cmp = Bytes.compareTo(r.metaEntry.getStartKey(),
590                 last.metaEntry.getEndKey());
591             if (cmp > 0) {
592               // hole
593               errors.reportError(ERROR_CODE.HOLE_IN_REGION_CHAIN,
594                   "There is a hole in the region chain.",
595                   this, r, last);
596             } else if (cmp < 0) {
597               // overlap
598               errors.reportError(ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
599                   "There is an overlap in the region chain.",
600                   this, r, last);
601             }
602           }
603 
604         }
605 
606         last = r;
607       }
608 
609       return errors.getErrorList().size() == 0;
610     }
611 
612   }
613 
614   /**
615    * Return a list of user-space table names whose metadata have not been
616    * modified in the last few milliseconds specified by timelag
617    * if any of the REGIONINFO_QUALIFIER, SERVER_QUALIFIER, STARTCODE_QUALIFIER,
618    * SPLITA_QUALIFIER, SPLITB_QUALIFIER have not changed in the last
619    * milliseconds specified by timelag, then the table is a candidate to be returned.
620    * @return tables that have not been modified recently
621    * @throws IOException if an error is encountered
622    */
623   HTableDescriptor[] getTables(AtomicInteger numSkipped) {
624     TreeSet<HTableDescriptor> uniqueTables = new TreeSet<HTableDescriptor>();
625     long now = System.currentTimeMillis();
626 
627     for (HbckInfo hbi : regionInfo.values()) {
628       MetaEntry info = hbi.metaEntry;
629 
630       // if the start key is zero, then we have found the first region of a table.
631       // pick only those tables that were not modified in the last few milliseconds.
632       if (info != null && info.getStartKey().length == 0 && !info.isMetaRegion()) {
633         if (info.modTime + timelag < now) {
634           uniqueTables.add(info.getTableDesc());
635         } else {
636           numSkipped.incrementAndGet(); // one more in-flux table
637         }
638       }
639     }
640     return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
641   }
642 
643   /**
644    * Gets the entry in regionInfo corresponding to the the given encoded
645    * region name. If the region has not been seen yet, a new entry is added
646    * and returned.
647    */
648   private synchronized HbckInfo getOrCreateInfo(String name) {
649     HbckInfo hbi = regionInfo.get(name);
650     if (hbi == null) {
651       hbi = new HbckInfo(null);
652       regionInfo.put(name, hbi);
653     }
654     return hbi;
655   }
656 
657   /**
658     * Check values in regionInfo for .META.
659     * Check if zero or more than one regions with META are found.
660     * If there are inconsistencies (i.e. zero or more than one regions
661     * pretend to be holding the .META.) try to fix that and report an error.
662     * @throws IOException from HBaseFsckRepair functions
663    * @throws KeeperException
664    * @throws InterruptedException
665     */
666   boolean checkMetaEntries()
667   throws IOException, KeeperException, InterruptedException {
668     List <HbckInfo> metaRegions = Lists.newArrayList();
669     for (HbckInfo value : regionInfo.values()) {
670       if (value.metaEntry.isMetaTable()) {
671         metaRegions.add(value);
672       }
673     }
674 
675     // If something is wrong
676     if (metaRegions.size() != 1) {
677       HRegionLocation rootLocation = connection.locateRegion(
678         HConstants.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW);
679       HbckInfo root =
680           regionInfo.get(rootLocation.getRegionInfo().getEncodedName());
681 
682       // If there is no region holding .META.
683       if (metaRegions.size() == 0) {
684         errors.reportError(ERROR_CODE.NO_META_REGION, ".META. is not found on any region.");
685         if (shouldFix()) {
686           errors.print("Trying to fix a problem with .META...");
687           setShouldRerun();
688           // try to fix it (treat it as unassigned region)
689           HBaseFsckRepair.fixUnassigned(conf, root.metaEntry);
690         }
691       }
692       // If there are more than one regions pretending to hold the .META.
693       else if (metaRegions.size() > 1) {
694         errors.reportError(ERROR_CODE.MULTI_META_REGION, ".META. is found on more than one region.");
695         if (shouldFix()) {
696           errors.print("Trying to fix a problem with .META...");
697           setShouldRerun();
698           // try fix it (treat is a dupe assignment)
699           List <HServerAddress> deployedOn = Lists.newArrayList();
700           for (HbckInfo mRegion : metaRegions) {
701             deployedOn.add(mRegion.metaEntry.regionServer);
702           }
703           HBaseFsckRepair.fixDupeAssignment(conf, root.metaEntry, deployedOn);
704         }
705       }
706       // rerun hbck with hopefully fixed META
707       return false;
708     }
709     // no errors, so continue normally
710     return true;
711   }
712 
713   /**
714    * Scan .META. and -ROOT-, adding all regions found to the regionInfo map.
715    * @throws IOException if an error is encountered
716    */
717   void getMetaEntries() throws IOException {
718     MetaScannerVisitor visitor = new MetaScannerVisitor() {
719       int countRecord = 1;
720 
721       // comparator to sort KeyValues with latest modtime
722       final Comparator<KeyValue> comp = new Comparator<KeyValue>() {
723         public int compare(KeyValue k1, KeyValue k2) {
724           return (int)(k1.getTimestamp() - k2.getTimestamp());
725         }
726       };
727 
728       public boolean processRow(Result result) throws IOException {
729         try {
730 
731           // record the latest modification of this META record
732           long ts =  Collections.max(result.list(), comp).getTimestamp();
733 
734           // record region details
735           byte [] value = result.getValue(HConstants.CATALOG_FAMILY,
736             HConstants.REGIONINFO_QUALIFIER);
737           if (value == null || value.length == 0) {
738             emptyRegionInfoQualifiers.add(result);
739             return true;
740           }
741           HRegionInfo info = Writables.getHRegionInfo(value);
742           HServerAddress server = null;
743           byte[] startCode = null;
744 
745           // record assigned region server
746           value = result.getValue(HConstants.CATALOG_FAMILY,
747                                      HConstants.SERVER_QUALIFIER);
748           if (value != null && value.length > 0) {
749             String address = Bytes.toString(value);
750             server = new HServerAddress(address);
751           }
752 
753           // record region's start key
754           value = result.getValue(HConstants.CATALOG_FAMILY,
755                                   HConstants.STARTCODE_QUALIFIER);
756           if (value != null) {
757             startCode = value;
758           }
759           MetaEntry m = new MetaEntry(info, server, startCode, ts);
760           HbckInfo hbInfo = new HbckInfo(m);
761           HbckInfo previous = regionInfo.put(info.getEncodedName(), hbInfo);
762           if (previous != null) {
763             throw new IOException("Two entries in META are same " + previous);
764           }
765 
766           // show proof of progress to the user, once for every 100 records.
767           if (countRecord % 100 == 0) {
768             errors.progress();
769           }
770           countRecord++;
771           return true;
772         } catch (RuntimeException e) {
773           LOG.error("Result=" + result);
774           throw e;
775         }
776       }
777     };
778 
779     // Scan -ROOT- to pick up META regions
780     MetaScanner.metaScan(conf, visitor, null, null,
781       Integer.MAX_VALUE, HConstants.ROOT_TABLE_NAME);
782 
783     // Scan .META. to pick up user regions
784     MetaScanner.metaScan(conf, visitor);
785     errors.print("");
786   }
787 
788   /**
789    * Stores the entries scanned from META
790    */
791   private static class MetaEntry extends HRegionInfo {
792     HServerAddress regionServer;   // server hosting this region
793     long modTime;          // timestamp of most recent modification metadata
794 
795     public MetaEntry(HRegionInfo rinfo, HServerAddress regionServer,
796                      byte[] startCode, long modTime) {
797       super(rinfo);
798       this.regionServer = regionServer;
799       this.modTime = modTime;
800     }
801   }
802 
803   /**
804    * Maintain information about a particular region.
805    */
806   static class HbckInfo implements Comparable {
807     boolean onlyEdits = false;
808     MetaEntry metaEntry = null;
809     FileStatus foundRegionDir = null;
810     List<HServerAddress> deployedOn = Lists.newArrayList();
811 
812     HbckInfo(MetaEntry metaEntry) {
813       this.metaEntry = metaEntry;
814     }
815 
816     public synchronized void addServer(HServerAddress server) {
817       this.deployedOn.add(server);
818     }
819 
820     public synchronized String toString() {
821       if (metaEntry != null) {
822         return metaEntry.getRegionNameAsString();
823       } else if (foundRegionDir != null) {
824         return foundRegionDir.getPath().toString();
825       } else {
826         return "UNKNOWN_REGION on " + Joiner.on(", ").join(deployedOn);
827       }
828     }
829 
830     @Override
831     public int compareTo(Object o) {
832       HbckInfo other = (HbckInfo) o;
833       int startComparison = Bytes.compareTo(this.metaEntry.getStartKey(), other.metaEntry.getStartKey());
834       if (startComparison != 0)
835         return startComparison;
836       else
837         return Bytes.compareTo(this.metaEntry.getEndKey(), other.metaEntry.getEndKey());
838     }
839   }
840 
841   /**
842    * Prints summary of all tables found on the system.
843    */
844   private void printTableSummary() {
845     System.out.println("Summary:");
846     for (TInfo tInfo : tablesInfo.values()) {
847       if (errors.tableHasErrors(tInfo)) {
848         System.out.println("Table " + tInfo.getName() + " is inconsistent.");
849       } else {
850         System.out.println("  " + tInfo.getName() + " is okay.");
851       }
852       System.out.println("    Number of regions: " + tInfo.getNumRegions());
853       System.out.print("    Deployed on: ");
854       for (HServerAddress server : tInfo.deployedOn) {
855         System.out.print(" " + server.toString());
856       }
857       System.out.println();
858     }
859   }
860 
861   interface ErrorReporter {
862     public static enum ERROR_CODE {
863       UNKNOWN, NO_META_REGION, NULL_ROOT_REGION, NO_VERSION_FILE, NOT_IN_META_HDFS, NOT_IN_META,
864       NOT_IN_META_OR_DEPLOYED, NOT_IN_HDFS_OR_DEPLOYED, NOT_IN_HDFS, SERVER_DOES_NOT_MATCH_META, NOT_DEPLOYED,
865       MULTI_DEPLOYED, SHOULD_NOT_BE_DEPLOYED, MULTI_META_REGION, RS_CONNECT_FAILURE,
866       FIRST_REGION_STARTKEY_NOT_EMPTY, DUPE_STARTKEYS,
867       HOLE_IN_REGION_CHAIN, OVERLAP_IN_REGION_CHAIN, REGION_CYCLE
868     }
869     public void clear();
870     public void report(String message);
871     public void reportError(String message);
872     public void reportError(ERROR_CODE errorCode, String message);
873     public void reportError(ERROR_CODE errorCode, String message, TInfo table, HbckInfo info);
874     public void reportError(ERROR_CODE errorCode, String message, TInfo table, HbckInfo info1, HbckInfo info2);
875     public int summarize();
876     public void detail(String details);
877     public ArrayList<ERROR_CODE> getErrorList();
878     public void progress();
879     public void print(String message);
880     public void resetErrors();
881     public boolean tableHasErrors(TInfo table);
882   }
883 
884   private static class PrintingErrorReporter implements ErrorReporter {
885     public int errorCount = 0;
886     private int showProgress;
887 
888     Set<TInfo> errorTables = new HashSet<TInfo>();
889 
890     // for use by unit tests to verify which errors were discovered
891     private ArrayList<ERROR_CODE> errorList = new ArrayList<ERROR_CODE>();
892 
893     public void clear() {
894       errorTables.clear();
895       errorList.clear();
896       errorCount = 0;
897     }
898 
899     public synchronized void reportError(ERROR_CODE errorCode, String message) {
900       errorList.add(errorCode);
901       if (!summary) {
902         System.out.println("ERROR: " + message);
903       }
904       errorCount++;
905       showProgress = 0;
906     }
907 
908     public synchronized void reportError(ERROR_CODE errorCode, String message, TInfo table,
909                                          HbckInfo info) {
910       errorTables.add(table);
911       String reference = "(region " + info.metaEntry.getRegionNameAsString() + ")";
912       reportError(errorCode, reference + " " + message);
913     }
914 
915     public synchronized void reportError(ERROR_CODE errorCode, String message, TInfo table,
916                                          HbckInfo info1, HbckInfo info2) {
917       errorTables.add(table);
918       String reference = "(regions " + info1.metaEntry.getRegionNameAsString()
919           + " and " + info2.metaEntry.getRegionNameAsString() + ")";
920       reportError(errorCode, reference + " " + message);
921     }
922 
923     public synchronized void reportError(String message) {
924       reportError(ERROR_CODE.UNKNOWN, message);
925     }
926 
927     /**
928      * Report error information, but do not increment the error count.  Intended for cases
929      * where the actual error would have been reported previously.
930      * @param message
931      */
932     public synchronized void report(String message) {
933       if (! summary) {
934         System.out.println("ERROR: " + message);
935       }
936       showProgress = 0;
937     }
938 
939     public synchronized int summarize() {
940       System.out.println(Integer.toString(errorCount) +
941                          " inconsistencies detected.");
942       if (errorCount == 0) {
943         System.out.println("Status: OK");
944         return 0;
945       } else {
946         System.out.println("Status: INCONSISTENT");
947         return -1;
948       }
949     }
950 
951     public ArrayList<ERROR_CODE> getErrorList() {
952       return errorList;
953     }
954 
955     public synchronized void print(String message) {
956       if (!summary) {
957         System.out.println(message);
958       }
959     }
960 
961     @Override
962     public boolean tableHasErrors(TInfo table) {
963       return errorTables.contains(table);
964     }
965 
966     @Override
967     public void resetErrors() {
968       errorCount = 0;
969     }
970 
971     public synchronized void detail(String message) {
972       if (details) {
973         System.out.println(message);
974       }
975       showProgress = 0;
976     }
977 
978     public synchronized void progress() {
979       if (showProgress++ == 10) {
980         if (!summary) {
981           System.out.print(".");
982         }
983         showProgress = 0;
984       }
985     }
986   }
987 
988   /**
989    * Contact a region server and get all information from it
990    */
991   static class WorkItemRegion implements Runnable {
992     private HBaseFsck hbck;
993     private HServerInfo rsinfo;
994     private ErrorReporter errors;
995     private HConnection connection;
996     private boolean done;
997 
998     WorkItemRegion(HBaseFsck hbck, HServerInfo info, 
999                    ErrorReporter errors, HConnection connection) {
1000       this.hbck = hbck;
1001       this.rsinfo = info;
1002       this.errors = errors;
1003       this.connection = connection;
1004       this.done = false;
1005     }
1006 
1007     // is this task done?
1008     synchronized boolean isDone() {
1009       return done;
1010     }
1011 
1012     @Override
1013     public synchronized void run() {
1014       errors.progress();
1015       try {
1016         HRegionInterface server = connection.getHRegionConnection(
1017                                     rsinfo.getServerAddress());
1018 
1019         // list all online regions from this region server
1020         List<HRegionInfo> regions = server.getOnlineRegions();
1021         if (details) {
1022           errors.detail("RegionServer: " + rsinfo.getServerName() +
1023                            " number of regions: " + regions.size());
1024           for (HRegionInfo rinfo: regions) {
1025             errors.detail("  " + rinfo.getRegionNameAsString() +
1026                              " id: " + rinfo.getRegionId() +
1027                              " encoded_name: " + rinfo.getEncodedName() +
1028                              " start: " + Bytes.toStringBinary(rinfo.getStartKey()) +
1029                              " end: " + Bytes.toStringBinary(rinfo.getEndKey()));
1030           }
1031         }
1032 
1033         // check to see if the existence of this region matches the region in META
1034         for (HRegionInfo r:regions) {
1035           HbckInfo hbi = hbck.getOrCreateInfo(r.getEncodedName());
1036           hbi.addServer(rsinfo.getServerAddress());
1037         }
1038       } catch (IOException e) {          // unable to connect to the region server. 
1039         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "RegionServer: " + rsinfo.getServerName() +
1040                       " Unable to fetch region information. " + e);
1041       } finally {
1042         done = true;
1043         notifyAll(); // wakeup anybody waiting for this item to be done
1044       }
1045     }
1046   }
1047 
1048   /**
1049    * Contact hdfs and get all information about spcified table directory.
1050    */
1051   static class WorkItemHdfsDir implements Runnable {
1052     private HBaseFsck hbck;
1053     private FileStatus tableDir;
1054     private ErrorReporter errors;
1055     private FileSystem fs;
1056     private boolean done;
1057 
1058     WorkItemHdfsDir(HBaseFsck hbck, FileSystem fs, ErrorReporter errors, 
1059                     FileStatus status) {
1060       this.hbck = hbck;
1061       this.fs = fs;
1062       this.tableDir = status;
1063       this.errors = errors;
1064       this.done = false;
1065     }
1066 
1067     synchronized boolean isDone() {
1068       return done;
1069     } 
1070 
1071     @Override
1072     public synchronized void run() {
1073       try {
1074         String tableName = tableDir.getPath().getName();
1075         // ignore hidden files
1076         if (tableName.startsWith(".") &&
1077             !tableName.equals( Bytes.toString(HConstants.META_TABLE_NAME)))
1078           return;
1079         // level 2: <HBASE_DIR>/<table>/*
1080         FileStatus[] regionDirs = fs.listStatus(tableDir.getPath());
1081         for (FileStatus regionDir : regionDirs) {
1082           String encodedName = regionDir.getPath().getName();
1083 
1084           // ignore directories that aren't hexadecimal
1085           if (!encodedName.toLowerCase().matches("[0-9a-f]+")) continue;
1086   
1087           HbckInfo hbi = hbck.getOrCreateInfo(encodedName);
1088           synchronized (hbi) {
1089             if (hbi.foundRegionDir != null) {
1090               errors.print("Directory " + encodedName + " duplicate??" +
1091                            hbi.foundRegionDir);
1092             }
1093             hbi.foundRegionDir = regionDir;
1094         
1095             // Set a flag if this region contains only edits
1096             // This is special case if a region is left after split
1097             hbi.onlyEdits = true;
1098             FileStatus[] subDirs = fs.listStatus(regionDir.getPath());
1099             Path ePath = HLog.getRegionDirRecoveredEditsDir(regionDir.getPath());
1100             for (FileStatus subDir : subDirs) {
1101               String sdName = subDir.getPath().getName();
1102               if (!sdName.startsWith(".") && !sdName.equals(ePath.getName())) {
1103                 hbi.onlyEdits = false;
1104                 break;
1105               }
1106             }
1107           }
1108         }
1109       } catch (IOException e) {          // unable to connect to the region server. 
1110         errors.reportError(ERROR_CODE.RS_CONNECT_FAILURE, "Table Directory: " + tableDir.getPath().getName() +
1111                       " Unable to fetch region information. " + e);
1112       } finally {
1113         done = true;
1114         notifyAll();
1115       }
1116     }
1117   }
1118 
1119   /**
1120    * Display the full report from fsck.
1121    * This displays all live and dead region servers, and all known regions.
1122    */
1123   void displayFullReport() {
1124     details = true;
1125   }
1126 
1127   /**
1128    * Set summary mode.
1129    * Print only summary of the tables and status (OK or INCONSISTENT)
1130    */
1131   void setSummary() {
1132     summary = true;
1133   }
1134 
1135   /**
1136    * Check if we should rerun fsck again. This checks if we've tried to
1137    * fix something and we should rerun fsck tool again.
1138    * Display the full report from fsck. This displays all live and dead
1139    * region servers, and all known regions.
1140    */
1141   void setShouldRerun() {
1142     rerun = true;
1143   }
1144 
1145   boolean shouldRerun() {
1146     return rerun;
1147   }
1148 
1149   /**
1150    * Fix inconsistencies found by fsck. This should try to fix errors (if any)
1151    * found by fsck utility.
1152    */
1153   void setFixErrors(boolean shouldFix) {
1154     fix = shouldFix;
1155   }
1156 
1157   boolean shouldFix() {
1158     return fix;
1159   }
1160 
1161   /**
1162    * We are interested in only those tables that have not changed their state in
1163    * META during the last few seconds specified by hbase.admin.fsck.timelag
1164    * @param seconds - the time in seconds
1165    */
1166   void setTimeLag(long seconds) {
1167     timelag = seconds * 1000; // convert to milliseconds
1168   }
1169 
1170   protected static void printUsageAndExit() {
1171     System.err.println("Usage: fsck [opts] ");
1172     System.err.println(" where [opts] are:");
1173     System.err.println("   -details Display full report of all regions.");
1174     System.err.println("   -timelag {timeInSeconds}  Process only regions that " +
1175                        " have not experienced any metadata updates in the last " +
1176                        " {{timeInSeconds} seconds.");
1177     System.err.println("   -fix Try to fix some of the errors.");
1178     System.err.println("   -sleepBeforeRerun {timeInSeconds} Sleep this many seconds" +
1179                        " before checking if the fix worked if run with -fix");
1180     System.err.println("   -summary Print only summary of the tables and status.");
1181 
1182     Runtime.getRuntime().exit(-2);
1183   }
1184 
1185   /**
1186    * Main program
1187    * @param args
1188    * @throws Exception
1189    */
1190   public static void main(String [] args) throws Exception {
1191 
1192     // create a fsck object
1193     Configuration conf = HBaseConfiguration.create();
1194     conf.set("fs.defaultFS", conf.get("hbase.rootdir"));
1195     HBaseFsck fsck = new HBaseFsck(conf);
1196     long sleepBeforeRerun = DEFAULT_SLEEP_BEFORE_RERUN;
1197 
1198     // Process command-line args.
1199     for (int i = 0; i < args.length; i++) {
1200       String cmd = args[i];
1201       if (cmd.equals("-details")) {
1202         fsck.displayFullReport();
1203       } else if (cmd.equals("-timelag")) {
1204         if (i == args.length - 1) {
1205           System.err.println("HBaseFsck: -timelag needs a value.");
1206           printUsageAndExit();
1207         }
1208         try {
1209           long timelag = Long.parseLong(args[i+1]);
1210           fsck.setTimeLag(timelag);
1211         } catch (NumberFormatException e) {
1212           System.err.println("-timelag needs a numeric value.");
1213           printUsageAndExit();
1214         }
1215         i++;
1216       } else if (cmd.equals("-sleepBeforeRerun")) {
1217         if (i == args.length - 1) {
1218           System.err.println("HBaseFsck: -sleepBeforeRerun needs a value.");
1219           printUsageAndExit();
1220         }
1221         try {
1222           sleepBeforeRerun = Long.parseLong(args[i+1]);
1223         } catch (NumberFormatException e) {
1224           System.err.println("-sleepBeforeRerun needs a numeric value.");
1225           printUsageAndExit();
1226         }
1227         i++;
1228       } else if (cmd.equals("-fix")) {
1229         fsck.setFixErrors(true);
1230       } else if (cmd.equals("-summary")) {
1231         fsck.setSummary();
1232       } else {
1233         String str = "Unknown command line option : " + cmd;
1234         LOG.info(str);
1235         System.out.println(str);
1236         printUsageAndExit();
1237       }
1238     }
1239     // do the real work of fsck
1240     int code = fsck.doWork();
1241     // If we have changed the HBase state it is better to run fsck again
1242     // to see if we haven't broken something else in the process.
1243     // We run it only once more because otherwise we can easily fall into
1244     // an infinite loop.
1245     if (fsck.shouldRerun()) {
1246       try {
1247         LOG.info("Sleeping " + sleepBeforeRerun + "ms before re-checking after fix...");
1248         Thread.sleep(sleepBeforeRerun);
1249       } catch (InterruptedException ie) {
1250         Runtime.getRuntime().exit(code);
1251       }
1252       // Just report
1253       fsck.setFixErrors(false);
1254       fsck.errors.resetErrors();
1255       code = fsck.doWork();
1256     }
1257 
1258     Runtime.getRuntime().exit(code);
1259   }
1260 }
1261