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