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