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 static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
23  import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
24  import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
25  import static org.junit.Assert.assertEquals;
26  import static org.junit.Assert.assertFalse;
27  import static org.junit.Assert.assertNotNull;
28  import static org.junit.Assert.assertTrue;
29  import static org.junit.Assert.fail;
30  
31  import java.io.IOException;
32  import java.util.ArrayList;
33  import java.util.Collection;
34  import java.util.HashMap;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.Map.Entry;
38  import java.util.concurrent.ExecutorService;
39  import java.util.concurrent.ScheduledThreadPoolExecutor;
40  
41  import org.apache.commons.io.IOUtils;
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.fs.FileStatus;
46  import org.apache.hadoop.fs.FileSystem;
47  import org.apache.hadoop.fs.Path;
48  import org.apache.hadoop.hbase.ClusterStatus;
49  import org.apache.hadoop.hbase.HBaseTestingUtility;
50  import org.apache.hadoop.hbase.HColumnDescriptor;
51  import org.apache.hadoop.hbase.HConstants;
52  import org.apache.hadoop.hbase.HRegionInfo;
53  import org.apache.hadoop.hbase.HRegionLocation;
54  import org.apache.hadoop.hbase.HTableDescriptor;
55  import org.apache.hadoop.hbase.LargeTests;
56  import org.apache.hadoop.hbase.MiniHBaseCluster;
57  import org.apache.hadoop.hbase.ServerName;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.client.Delete;
60  import org.apache.hadoop.hbase.client.Get;
61  import org.apache.hadoop.hbase.client.HBaseAdmin;
62  import org.apache.hadoop.hbase.client.HConnection;
63  import org.apache.hadoop.hbase.client.HTable;
64  import org.apache.hadoop.hbase.client.Put;
65  import org.apache.hadoop.hbase.client.Result;
66  import org.apache.hadoop.hbase.client.ResultScanner;
67  import org.apache.hadoop.hbase.client.Scan;
68  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
69  import org.apache.hadoop.hbase.executor.RegionTransitionData;
70  import org.apache.hadoop.hbase.io.hfile.TestHFile;
71  import org.apache.hadoop.hbase.ipc.HRegionInterface;
72  import org.apache.hadoop.hbase.master.HMaster;
73  import org.apache.hadoop.hbase.regionserver.HRegion;
74  import org.apache.hadoop.hbase.regionserver.HRegionServer;
75  import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
76  import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
77  import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
78  import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
79  import org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter;
80  import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
81  import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
82  import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
83  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
84  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
85  import org.apache.zookeeper.KeeperException;
86  import org.junit.AfterClass;
87  import org.junit.BeforeClass;
88  import org.junit.Test;
89  import org.junit.experimental.categories.Category;
90  import org.junit.rules.TestName;
91  
92  import com.google.common.collect.Multimap;
93  
94  /**
95   * This tests HBaseFsck's ability to detect reasons for inconsistent tables.
96   */
97  @Category(LargeTests.class)
98  public class TestHBaseFsck {
99    final static Log LOG = LogFactory.getLog(TestHBaseFsck.class);
100   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
101   private final static Configuration conf = TEST_UTIL.getConfiguration();
102   private final static String FAM_STR = "fam";
103   private final static byte[] FAM = Bytes.toBytes(FAM_STR);
104   private final static int REGION_ONLINE_TIMEOUT = 800;
105 
106   // for the instance, reset every test run
107   private HTable tbl;
108   private final static byte[][] SPLITS = new byte[][] { Bytes.toBytes("A"),
109     Bytes.toBytes("B"), Bytes.toBytes("C") };
110   // one row per region.
111   private final static byte[][] ROWKEYS= new byte[][] {
112     Bytes.toBytes("00"), Bytes.toBytes("50"), Bytes.toBytes("A0"), Bytes.toBytes("A5"),
113     Bytes.toBytes("B0"), Bytes.toBytes("B5"), Bytes.toBytes("C0"), Bytes.toBytes("C5") };
114 
115   @BeforeClass
116   public static void setUpBeforeClass() throws Exception {
117     TEST_UTIL.getConfiguration().setBoolean("hbase.master.distributed.log.splitting", false);
118     TEST_UTIL.startMiniCluster(3);
119   }
120 
121   @AfterClass
122   public static void tearDownAfterClass() throws Exception {
123     TEST_UTIL.shutdownMiniCluster();
124   }
125 
126   @Test
127   public void testHBaseFsck() throws Exception {
128     assertNoErrors(doFsck(conf, false));
129     String table = "tableBadMetaAssign"; 
130     TEST_UTIL.createTable(Bytes.toBytes(table), FAM);
131 
132     // We created 1 table, should be fine
133     assertNoErrors(doFsck(conf, false));
134 
135     // Now let's mess it up and change the assignment in .META. to
136     // point to a different region server
137     HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
138     ResultScanner scanner = meta.getScanner(new Scan());
139 
140     resforloop:
141     for (Result res : scanner) {
142       long startCode = Bytes.toLong(res.getValue(HConstants.CATALOG_FAMILY,
143           HConstants.STARTCODE_QUALIFIER));
144 
145       for (JVMClusterUtil.RegionServerThread rs :
146           TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
147 
148         ServerName sn = rs.getRegionServer().getServerName();
149 
150         // When we find a diff RS, change the assignment and break
151         if (startCode != sn.getStartcode()) {
152           Put put = new Put(res.getRow());
153           put.setWriteToWAL(false);
154           put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
155             Bytes.toBytes(sn.getHostAndPort()));
156           put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
157             Bytes.toBytes(sn.getStartcode()));
158           meta.put(put);
159           break resforloop;
160         }
161       }
162     }
163 
164     // Try to fix the data
165     assertErrors(doFsck(conf, true), new ERROR_CODE[]{
166         ERROR_CODE.SERVER_DOES_NOT_MATCH_META});
167 
168     // fixing assignments require opening regions is not synchronous.  To make
169     // the test pass consistently so for now we bake in some sleep to let it
170     // finish.  1s seems sufficient.
171     Thread.sleep(1000);
172 
173     // Should be fixed now
174     assertNoErrors(doFsck(conf, false));
175 
176     // comment needed - what is the purpose of this line
177     HTable t = new HTable(conf, Bytes.toBytes(table));
178     ResultScanner s = t.getScanner(new Scan());
179     s.close();
180     t.close();
181 
182     scanner.close();
183     meta.close();
184   }
185 
186   /**
187    * Create a new region in META.
188    */
189   private HRegionInfo createRegion(Configuration conf, final HTableDescriptor
190       htd, byte[] startKey, byte[] endKey)
191       throws IOException {
192     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
193     HRegionInfo hri = new HRegionInfo(htd.getName(), startKey, endKey);
194     Put put = new Put(hri.getRegionName());
195     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
196         Writables.getBytes(hri));
197     meta.put(put);
198     return hri;
199   }
200 
201   /**
202    * Debugging method to dump the contents of meta.
203    */
204   private void dumpMeta(byte[] tableName) throws IOException {
205     List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(tableName);
206     for (byte[] row : metaRows) {
207       LOG.info(Bytes.toString(row));
208     }
209   }
210 
211   /**
212    * This method is used to undeploy a region -- close it and attempt to
213    * remove its state from the Master.
214    */
215   private void undeployRegion(HBaseAdmin admin, ServerName sn,
216       HRegionInfo hri) throws IOException, InterruptedException {
217     try {
218       HBaseFsckRepair.closeRegionSilentlyAndWait(admin, sn, hri);
219       admin.getMaster().offline(hri.getRegionName());
220     } catch (IOException ioe) {
221       LOG.warn("Got exception when attempting to offline region "
222           + Bytes.toString(hri.getRegionName()), ioe);
223     }
224   }
225   /**
226    * Delete a region from assignments, meta, or completely from hdfs.
227    * @param unassign if true unassign region if assigned
228    * @param metaRow  if true remove region's row from META
229    * @param hdfs if true remove region's dir in HDFS
230    */
231   private void deleteRegion(Configuration conf, final HTableDescriptor htd,
232       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
233       boolean hdfs) throws IOException, InterruptedException {
234     deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false);
235   }
236 
237   /**
238    * Delete a region from assignments, meta, or completely from hdfs.
239    * @param unassign if true unassign region if assigned
240    * @param metaRow  if true remove region's row from META
241    * @param hdfs if true remove region's dir in HDFS
242    * @param regionInfoOnly if true remove a region dir's .regioninfo file
243    */
244   private void deleteRegion(Configuration conf, final HTableDescriptor htd,
245       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
246       boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException {
247     LOG.info("** Before delete:");
248     dumpMeta(htd.getName());
249 
250     Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
251     for (Entry<HRegionInfo, ServerName> e: hris.entrySet()) {
252       HRegionInfo hri = e.getKey();
253       ServerName hsa = e.getValue();
254       if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
255           && Bytes.compareTo(hri.getEndKey(), endKey) == 0) {
256 
257         LOG.info("RegionName: " +hri.getRegionNameAsString());
258         byte[] deleteRow = hri.getRegionName();
259 
260         if (unassign) {
261           LOG.info("Undeploying region " + hri + " from server " + hsa);
262           undeployRegion(new HBaseAdmin(conf), hsa, new HRegionInfo(hri));
263         }
264 
265         if (regionInfoOnly) {
266           LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
267           Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
268           FileSystem fs = rootDir.getFileSystem(conf);
269           Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
270           Path hriPath = new Path(p, HRegion.REGIONINFO_FILE);
271           fs.delete(hriPath, true);
272         }
273 
274         if (hdfs) {
275           LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
276           Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
277           FileSystem fs = rootDir.getFileSystem(conf);
278           Path p = new Path(rootDir + "/" + htd.getNameAsString(), hri.getEncodedName());
279           HBaseFsck.debugLsr(conf, p);
280           boolean success = fs.delete(p, true);
281           LOG.info("Deleted " + p + " sucessfully? " + success);
282           HBaseFsck.debugLsr(conf, p);
283         }
284 
285         if (metaRow) {
286           HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
287           Delete delete = new Delete(deleteRow);
288           meta.delete(delete);
289         }
290       }
291       LOG.info(hri.toString() + hsa.toString());
292     }
293 
294     TEST_UTIL.getMetaTableRows(htd.getName());
295     LOG.info("*** After delete:");
296     dumpMeta(htd.getName());
297   }
298 
299   /**
300    * Setup a clean table before we start mucking with it.
301    * 
302    * @throws IOException
303    * @throws InterruptedException
304    * @throws KeeperException
305    */
306   HTable setupTable(String tablename) throws Exception {
307     HTableDescriptor desc = new HTableDescriptor(tablename);
308     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
309     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
310     TEST_UTIL.getHBaseAdmin().createTable(desc, SPLITS);
311     tbl = new HTable(TEST_UTIL.getConfiguration(), tablename);
312 
313     List<Put> puts = new ArrayList<Put>();
314     for (byte[] row : ROWKEYS) {
315       Put p = new Put(row);
316       p.add(FAM, Bytes.toBytes("val"), row);
317       puts.add(p);
318     }
319     tbl.put(puts);
320     tbl.flushCommits();
321     long endTime = System.currentTimeMillis() + 60000;
322     while (!TEST_UTIL.getHBaseAdmin().isTableEnabled(tablename)) {
323       try {
324         if (System.currentTimeMillis() > endTime) {
325           fail("Failed to enable table " + tablename + " after waiting for 60 sec");
326         }
327         Thread.sleep(100);
328       } catch (InterruptedException e) {
329         e.printStackTrace();
330         fail("Interrupted when waiting table " + tablename + " to be enabled");
331       }
332     }
333     return tbl;
334   }
335 
336   /**
337    * Counts the number of row to verify data loss or non-dataloss.
338    */
339   int countRows() throws IOException {
340      Scan s = new Scan();
341      ResultScanner rs = tbl.getScanner(s);
342      int i = 0;
343      while(rs.next() !=null) {
344        i++;
345      }
346      return i;
347   }
348 
349   /**
350    * delete table in preparation for next test
351    * 
352    * @param tablename
353    * @throws IOException
354    */
355   void deleteTable(String tablename) throws IOException {
356     HBaseAdmin admin = new HBaseAdmin(conf);
357     admin.getConnection().clearRegionCache();
358     byte[] tbytes = Bytes.toBytes(tablename);
359     admin.disableTableAsync(tbytes);
360     while (!admin.isTableDisabled(tbytes)) {
361       try {
362         Thread.sleep(250);
363       } catch (InterruptedException e) {
364         e.printStackTrace();
365         fail("Interrupted when trying to disable table " + tablename);
366       }
367     }
368     admin.deleteTable(tbytes);
369   }
370 
371   /**
372    * This creates a clean table and confirms that the table is clean.
373    */
374   @Test
375   public void testHBaseFsckClean() throws Exception {
376     assertNoErrors(doFsck(conf, false));
377     String table = "tableClean";
378     try {
379       HBaseFsck hbck = doFsck(conf, false);
380       assertNoErrors(hbck);
381 
382       setupTable(table);
383       assertEquals(ROWKEYS.length, countRows());
384 
385       // We created 1 table, should be fine
386       hbck = doFsck(conf, false);
387       assertNoErrors(hbck);
388       assertEquals(0, hbck.getOverlapGroups(table).size());
389       assertEquals(ROWKEYS.length, countRows());
390     } finally {
391       deleteTable(table);
392     }
393   }
394 
395   /**
396    * Test thread pooling in the case where there are more regions than threads
397    */
398   @Test
399   public void testHbckThreadpooling() throws Exception {
400     String table = "tableDupeStartKey";
401     try {
402       // Create table with 4 regions
403       setupTable(table);
404 
405       // limit number of threads to 1.
406       Configuration newconf = new Configuration(conf);
407       newconf.setInt("hbasefsck.numthreads", 1);  
408       assertNoErrors(doFsck(newconf, false));
409       
410       // We should pass without triggering a RejectedExecutionException
411     } finally {
412       deleteTable(table);
413     }    
414   }
415 
416   @Test
417   public void testHbckFixOrphanTable() throws Exception {
418     String table = "tableInfo";
419     FileSystem fs = null;
420     Path tableinfo = null;
421     try {
422       setupTable(table);
423       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
424 
425       Path hbaseTableDir = new Path(conf.get(HConstants.HBASE_DIR) + "/" + table );
426       fs = hbaseTableDir.getFileSystem(conf);
427       FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
428       tableinfo = status.getPath();
429       fs.rename(tableinfo, new Path("/.tableinfo"));
430 
431       //to report error if .tableinfo is missing.
432       HBaseFsck hbck = doFsck(conf, false);
433       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_TABLEINFO_FILE });
434 
435       // fix OrphanTable with default .tableinfo (htd not yet cached on master)
436       hbck = doFsck(conf, true);
437       assertNoErrors(hbck);
438       status = null;
439       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
440       assertNotNull(status);
441 
442       HTableDescriptor htd = admin.getTableDescriptor(table.getBytes());
443       htd.setValue("NOT_DEFAULT", "true");
444       admin.disableTable(table);
445       admin.modifyTable(table.getBytes(), htd);
446       admin.enableTable(table);
447       fs.delete(status.getPath(), true);
448 
449       // fix OrphanTable with cache
450       htd = admin.getTableDescriptor(table.getBytes()); // warms up cached htd on master
451       hbck = doFsck(conf, true);
452       assertNoErrors(hbck);
453       status = null;
454       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
455       assertNotNull(status);
456       htd = admin.getTableDescriptor(table.getBytes());
457       assertEquals(htd.getValue("NOT_DEFAULT"), "true");
458     } finally {
459       fs.rename(new Path("/.tableinfo"), tableinfo);
460       deleteTable(table);
461     }
462   }
463 
464   /**
465    * This create and fixes a bad table with regions that have a duplicate
466    * start key
467    */
468   @Test
469   public void testDupeStartKey() throws Exception {
470     String table = "tableDupeStartKey";
471     try {
472       setupTable(table);
473       assertNoErrors(doFsck(conf, false));
474       assertEquals(ROWKEYS.length, countRows());
475 
476       // Now let's mess it up, by adding a region with a duplicate startkey
477       HRegionInfo hriDupe = createRegion(conf, tbl.getTableDescriptor(),
478           Bytes.toBytes("A"), Bytes.toBytes("A2"));
479       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
480       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
481           .waitForAssignment(hriDupe);
482 
483       HBaseFsck hbck = doFsck(conf, false);
484       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
485             ERROR_CODE.DUPE_STARTKEYS});
486       assertEquals(2, hbck.getOverlapGroups(table).size());
487       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
488 
489       // fix the degenerate region.
490       doFsck(conf,true);
491 
492       // check that the degenerate region is gone and no data loss
493       HBaseFsck hbck2 = doFsck(conf,false);
494       assertNoErrors(hbck2);
495       assertEquals(0, hbck2.getOverlapGroups(table).size());
496       assertEquals(ROWKEYS.length, countRows());
497     } finally {
498       deleteTable(table);
499     }
500   }
501 
502   /**
503    * Get region info from local cluster.
504    */
505   Map<ServerName, List<String>> getDeployedHRIs(HBaseAdmin admin)
506     throws IOException {
507     ClusterStatus status = admin.getMaster().getClusterStatus();
508     Collection<ServerName> regionServers = status.getServers();
509     Map<ServerName, List<String>> mm =
510         new HashMap<ServerName, List<String>>();
511     HConnection connection = admin.getConnection();
512     for (ServerName hsi : regionServers) {
513       HRegionInterface server =
514         connection.getHRegionConnection(hsi.getHostname(), hsi.getPort());
515 
516       // list all online regions from this region server
517       List<HRegionInfo> regions = server.getOnlineRegions();
518       List<String> regionNames = new ArrayList<String>();
519       for (HRegionInfo hri : regions) {
520         regionNames.add(hri.getRegionNameAsString());
521       }
522       mm.put(hsi, regionNames);
523     }
524     return mm;
525   }
526 
527   /**
528    * Returns the HSI a region info is on.
529    */
530   ServerName findDeployedHSI(Map<ServerName, List<String>> mm, HRegionInfo hri) {
531     for (Map.Entry<ServerName,List <String>> e : mm.entrySet()) {
532       if (e.getValue().contains(hri.getRegionNameAsString())) {
533         return e.getKey();
534       }
535     }
536     return null;
537   }
538 
539   /**
540    * This create and fixes a bad table with regions that have a duplicate
541    * start key
542    */
543   @Test
544   public void testDupeRegion() throws Exception {
545     String table = "tableDupeRegion";
546     try {
547       setupTable(table);
548       assertNoErrors(doFsck(conf, false));
549       assertEquals(ROWKEYS.length, countRows());
550 
551       // Now let's mess it up, by adding a region with a duplicate startkey
552       HRegionInfo hriDupe = createRegion(conf, tbl.getTableDescriptor(),
553           Bytes.toBytes("A"), Bytes.toBytes("B"));
554 
555       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
556       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
557           .waitForAssignment(hriDupe);
558 
559       // Yikes! The assignment manager can't tell between diff between two
560       // different regions with the same start/endkeys since it doesn't
561       // differentiate on ts/regionId!  We actually need to recheck
562       // deployments!
563       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
564       while (findDeployedHSI(getDeployedHRIs(admin), hriDupe) == null) {
565         Thread.sleep(250);
566       }
567 
568       LOG.debug("Finished assignment of dupe region");
569 
570       // TODO why is dupe region different from dupe start keys?
571       HBaseFsck hbck = doFsck(conf, false);
572       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
573             ERROR_CODE.DUPE_STARTKEYS});
574       assertEquals(2, hbck.getOverlapGroups(table).size());
575       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
576 
577       // fix the degenerate region.
578       doFsck(conf,true);
579 
580       // check that the degenerate region is gone and no data loss
581       HBaseFsck hbck2 = doFsck(conf,false);
582       assertNoErrors(hbck2);
583       assertEquals(0, hbck2.getOverlapGroups(table).size());
584       assertEquals(ROWKEYS.length, countRows());
585     } finally {
586       deleteTable(table);
587     }
588   }
589 
590   /**
591    * This creates and fixes a bad table with regions that has startkey == endkey
592    */
593   @Test
594   public void testDegenerateRegions() throws Exception {
595     String table = "tableDegenerateRegions";
596     try {
597       setupTable(table);
598       assertNoErrors(doFsck(conf,false));
599       assertEquals(ROWKEYS.length, countRows());
600 
601       // Now let's mess it up, by adding a region with a duplicate startkey
602       HRegionInfo hriDupe = createRegion(conf, tbl.getTableDescriptor(),
603           Bytes.toBytes("B"), Bytes.toBytes("B"));
604       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
605       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
606           .waitForAssignment(hriDupe);
607 
608       HBaseFsck hbck = doFsck(conf,false);
609       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DEGENERATE_REGION,
610           ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS});
611       assertEquals(2, hbck.getOverlapGroups(table).size());
612       assertEquals(ROWKEYS.length, countRows());
613 
614       // fix the degenerate region.
615       doFsck(conf,true);
616 
617       // check that the degenerate region is gone and no data loss
618       HBaseFsck hbck2 = doFsck(conf,false);
619       assertNoErrors(hbck2);
620       assertEquals(0, hbck2.getOverlapGroups(table).size());
621       assertEquals(ROWKEYS.length, countRows());
622     } finally {
623       deleteTable(table);
624     }
625   }
626 
627   /**
628    * This creates and fixes a bad table where a region is completely contained
629    * by another region.
630    */
631   @Test
632   public void testContainedRegionOverlap() throws Exception {
633     String table = "tableContainedRegionOverlap";
634     try {
635       setupTable(table);
636       assertEquals(ROWKEYS.length, countRows());
637 
638       // Mess it up by creating an overlap in the metadata
639       HRegionInfo hriOverlap = createRegion(conf, tbl.getTableDescriptor(),
640           Bytes.toBytes("A2"), Bytes.toBytes("B"));
641       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
642       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
643           .waitForAssignment(hriOverlap);
644 
645       HBaseFsck hbck = doFsck(conf, false);
646       assertErrors(hbck, new ERROR_CODE[] {
647           ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
648       assertEquals(2, hbck.getOverlapGroups(table).size());
649       assertEquals(ROWKEYS.length, countRows());
650 
651       // fix the problem.
652       doFsck(conf, true);
653 
654       // verify that overlaps are fixed
655       HBaseFsck hbck2 = doFsck(conf,false);
656       assertNoErrors(hbck2);
657       assertEquals(0, hbck2.getOverlapGroups(table).size());
658       assertEquals(ROWKEYS.length, countRows());
659     } finally {
660        deleteTable(table);
661     }
662   }
663 
664   /**
665    * This creates and fixes a bad table where an overlap group of
666    * 3 regions. Set HBaseFsck.maxMerge to 2 to trigger sideline overlapped
667    * region. Mess around the meta data so that closeRegion/offlineRegion
668    * throws exceptions.
669    */
670   @Test
671   public void testSidelineOverlapRegion() throws Exception {
672     String table = "testSidelineOverlapRegion";
673     try {
674       setupTable(table);
675       assertEquals(ROWKEYS.length, countRows());
676 
677       // Mess it up by creating an overlap
678       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
679       HMaster master = cluster.getMaster();
680       HRegionInfo hriOverlap1 = createRegion(conf, tbl.getTableDescriptor(),
681         Bytes.toBytes("A"), Bytes.toBytes("AB"));
682       master.assignRegion(hriOverlap1);
683       master.getAssignmentManager().waitForAssignment(hriOverlap1);
684       HRegionInfo hriOverlap2 = createRegion(conf, tbl.getTableDescriptor(),
685         Bytes.toBytes("AB"), Bytes.toBytes("B"));
686       master.assignRegion(hriOverlap2);
687       master.getAssignmentManager().waitForAssignment(hriOverlap2);
688 
689       HBaseFsck hbck = doFsck(conf, false);
690       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.DUPE_STARTKEYS,
691         ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.OVERLAP_IN_REGION_CHAIN});
692       assertEquals(3, hbck.getOverlapGroups(table).size());
693       assertEquals(ROWKEYS.length, countRows());
694 
695       // mess around the overlapped regions, to trigger NotServingRegionException
696       Multimap<byte[], HbckInfo> overlapGroups = hbck.getOverlapGroups(table);
697       ServerName serverName = null;
698       byte[] regionName = null;
699       for (HbckInfo hbi: overlapGroups.values()) {
700         if ("A".equals(Bytes.toString(hbi.getStartKey()))
701             && "B".equals(Bytes.toString(hbi.getEndKey()))) {
702           regionName = hbi.getRegionName();
703 
704           // get an RS not serving the region to force bad assignment info in to META.
705           int k = cluster.getServerWith(regionName);
706           for (int i = 0; i < 3; i++) {
707             if (i != k) {
708               HRegionServer rs = cluster.getRegionServer(i);
709               serverName = rs.getServerName();
710               break;
711             }
712           }
713 
714           HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
715           HBaseFsckRepair.closeRegionSilentlyAndWait(admin,
716             cluster.getRegionServer(k).getServerName(), hbi.getHdfsHRI());
717           admin.unassign(regionName, true);
718           break;
719         }
720       }
721 
722       assertNotNull(regionName);
723       assertNotNull(serverName);
724       HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
725       Put put = new Put(regionName);
726       put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
727         Bytes.toBytes(serverName.getHostAndPort()));
728       meta.put(put);
729 
730       // fix the problem.
731       HBaseFsck fsck = new HBaseFsck(conf);
732       fsck.connect();
733       fsck.setDisplayFullReport(); // i.e. -details
734       fsck.setTimeLag(0);
735       fsck.setFixAssignments(true);
736       fsck.setFixMeta(true);
737       fsck.setFixHdfsHoles(true);
738       fsck.setFixHdfsOverlaps(true);
739       fsck.setFixHdfsOrphans(true);
740       fsck.setFixVersionFile(true);
741       fsck.setSidelineBigOverlaps(true);
742       fsck.setMaxMerge(2);
743       fsck.onlineHbck();
744 
745       // verify that overlaps are fixed, and there are less rows
746       // since one region is sidelined.
747       HBaseFsck hbck2 = doFsck(conf,false);
748       assertNoErrors(hbck2);
749       assertEquals(0, hbck2.getOverlapGroups(table).size());
750       assertTrue(ROWKEYS.length > countRows());
751     } finally {
752        deleteTable(table);
753     }
754   }
755 
756   /**
757    * This creates and fixes a bad table where a region is completely contained
758    * by another region, and there is a hole (sort of like a bad split)
759    */
760   @Test
761   public void testOverlapAndOrphan() throws Exception {
762     String table = "tableOverlapAndOrphan";
763     try {
764       setupTable(table);
765       assertEquals(ROWKEYS.length, countRows());
766 
767       // Mess it up by creating an overlap in the metadata
768       TEST_UTIL.getHBaseAdmin().disableTable(table);
769       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
770           Bytes.toBytes("B"), true, true, false, true);
771       TEST_UTIL.getHBaseAdmin().enableTable(table);
772 
773       HRegionInfo hriOverlap = createRegion(conf, tbl.getTableDescriptor(),
774           Bytes.toBytes("A2"), Bytes.toBytes("B"));
775       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
776       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
777           .waitForAssignment(hriOverlap);
778 
779       HBaseFsck hbck = doFsck(conf, false);
780       assertErrors(hbck, new ERROR_CODE[] {
781           ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
782           ERROR_CODE.HOLE_IN_REGION_CHAIN});
783 
784       // fix the problem.
785       doFsck(conf, true);
786 
787       // verify that overlaps are fixed
788       HBaseFsck hbck2 = doFsck(conf,false);
789       assertNoErrors(hbck2);
790       assertEquals(0, hbck2.getOverlapGroups(table).size());
791       assertEquals(ROWKEYS.length, countRows());
792     } finally {
793        deleteTable(table);
794     }
795   }
796 
797   /**
798    * This creates and fixes a bad table where a region overlaps two regions --
799    * a start key contained in another region and its end key is contained in
800    * yet another region.
801    */
802   @Test
803   public void testCoveredStartKey() throws Exception {
804     String table = "tableCoveredStartKey";
805     try {
806       setupTable(table);
807       assertEquals(ROWKEYS.length, countRows());
808 
809       // Mess it up by creating an overlap in the metadata
810       HRegionInfo hriOverlap = createRegion(conf, tbl.getTableDescriptor(),
811           Bytes.toBytes("A2"), Bytes.toBytes("B2"));
812       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
813       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
814           .waitForAssignment(hriOverlap);
815 
816       HBaseFsck hbck = doFsck(conf, false);
817       assertErrors(hbck, new ERROR_CODE[] {
818           ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
819           ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
820       assertEquals(3, hbck.getOverlapGroups(table).size());
821       assertEquals(ROWKEYS.length, countRows());
822 
823       // fix the problem.
824       doFsck(conf, true);
825 
826       // verify that overlaps are fixed
827       HBaseFsck hbck2 = doFsck(conf, false);
828       assertErrors(hbck2, new ERROR_CODE[0]);
829       assertEquals(0, hbck2.getOverlapGroups(table).size());
830       assertEquals(ROWKEYS.length, countRows());
831     } finally {
832       deleteTable(table);
833     }
834   }
835 
836   /**
837    * This creates and fixes a bad table with a missing region -- hole in meta
838    * and data missing in the fs.
839    */
840   @Test
841   public void testRegionHole() throws Exception {
842     String table = "tableRegionHole";
843     try {
844       setupTable(table);
845       assertEquals(ROWKEYS.length, countRows());
846 
847       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
848       TEST_UTIL.getHBaseAdmin().disableTable(table);
849       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
850           Bytes.toBytes("C"), true, true, true);
851       TEST_UTIL.getHBaseAdmin().enableTable(table);
852 
853       HBaseFsck hbck = doFsck(conf, false);
854       assertErrors(hbck, new ERROR_CODE[] {
855           ERROR_CODE.HOLE_IN_REGION_CHAIN});
856       // holes are separate from overlap groups
857       assertEquals(0, hbck.getOverlapGroups(table).size());
858 
859       // fix hole
860       doFsck(conf, true);
861 
862       // check that hole fixed
863       assertNoErrors(doFsck(conf,false));
864       assertEquals(ROWKEYS.length - 2 , countRows()); // lost a region so lost a row
865     } finally {
866       deleteTable(table);
867     }
868   }
869 
870   /**
871    * This creates and fixes a bad table with a missing region -- hole in meta
872    * and data present but .regioinfino missing (an orphan hdfs region)in the fs.
873    */
874   @Test
875   public void testHDFSRegioninfoMissing() throws Exception {
876     String table = "tableHDFSRegioininfoMissing";
877     try {
878       setupTable(table);
879       assertEquals(ROWKEYS.length, countRows());
880 
881       // Mess it up by leaving a hole in the meta data
882       TEST_UTIL.getHBaseAdmin().disableTable(table);
883       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
884           Bytes.toBytes("C"), true, true, false, true);
885       TEST_UTIL.getHBaseAdmin().enableTable(table);
886 
887       HBaseFsck hbck = doFsck(conf, false);
888       assertErrors(hbck, new ERROR_CODE[] {
889           ERROR_CODE.ORPHAN_HDFS_REGION,
890           ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
891           ERROR_CODE.HOLE_IN_REGION_CHAIN});
892       // holes are separate from overlap groups
893       assertEquals(0, hbck.getOverlapGroups(table).size());
894 
895       // fix hole
896       doFsck(conf, true);
897 
898       // check that hole fixed
899       assertNoErrors(doFsck(conf, false));
900       assertEquals(ROWKEYS.length, countRows());
901     } finally {
902       deleteTable(table);
903     }
904   }
905 
906   /**
907    * This creates and fixes a bad table with a region that is missing meta and
908    * not assigned to a region server.
909    */
910   @Test
911   public void testNotInMetaOrDeployedHole() throws Exception {
912     String table = "tableNotInMetaOrDeployedHole";
913     try {
914       setupTable(table);
915       assertEquals(ROWKEYS.length, countRows());
916 
917       // Mess it up by leaving a hole in the meta data
918       TEST_UTIL.getHBaseAdmin().disableTable(table);
919       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
920           Bytes.toBytes("C"), true, true, false); // don't rm from fs
921       TEST_UTIL.getHBaseAdmin().enableTable(table);
922 
923       HBaseFsck hbck = doFsck(conf, false);
924       assertErrors(hbck, new ERROR_CODE[] {
925           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
926       // holes are separate from overlap groups
927       assertEquals(0, hbck.getOverlapGroups(table).size());
928 
929       // fix hole
930       assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
931           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
932 
933       // check that hole fixed
934       assertNoErrors(doFsck(conf,false));
935       assertEquals(ROWKEYS.length, countRows());
936     } finally {
937       deleteTable(table);
938     }
939   }
940 
941   /**
942    * This creates fixes a bad table with a hole in meta.
943    */
944   @Test
945   public void testNotInMetaHole() throws Exception {
946     String table = "tableNotInMetaHole";
947     try {
948       setupTable(table);
949       assertEquals(ROWKEYS.length, countRows());
950 
951       // Mess it up by leaving a hole in the meta data
952       TEST_UTIL.getHBaseAdmin().disableTable(table);
953       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
954           Bytes.toBytes("C"), false, true, false); // don't rm from fs
955       TEST_UTIL.getHBaseAdmin().enableTable(table);
956 
957       HBaseFsck hbck = doFsck(conf, false);
958       assertErrors(hbck, new ERROR_CODE[] {
959           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
960       // holes are separate from overlap groups
961       assertEquals(0, hbck.getOverlapGroups(table).size());
962 
963       // fix hole
964       assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
965           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
966 
967       // check that hole fixed
968       assertNoErrors(doFsck(conf,false));
969       assertEquals(ROWKEYS.length, countRows());
970     } finally {
971       deleteTable(table);
972     }
973   }
974 
975   /**
976    * This creates and fixes a bad table with a region that is in meta but has
977    * no deployment or data hdfs
978    */
979   @Test
980   public void testNotInHdfs() throws Exception {
981     String table = "tableNotInHdfs";
982     try {
983       setupTable(table);
984       assertEquals(ROWKEYS.length, countRows());
985 
986       // make sure data in regions, if in hlog only there is no data loss
987       TEST_UTIL.getHBaseAdmin().flush(table);
988 
989       // Mess it up by leaving a hole in the hdfs data
990       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
991           Bytes.toBytes("C"), false, false, true); // don't rm meta
992 
993       HBaseFsck hbck = doFsck(conf, false);
994       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
995       // holes are separate from overlap groups
996       assertEquals(0, hbck.getOverlapGroups(table).size());
997 
998       // fix hole
999       doFsck(conf, true);
1000 
1001       // check that hole fixed
1002       assertNoErrors(doFsck(conf,false));
1003       assertEquals(ROWKEYS.length - 2, countRows());
1004     } finally {
1005       deleteTable(table);
1006     }
1007   }
1008 
1009   /**
1010    * This creates entries in META with no hdfs data.  This should cleanly
1011    * remove the table.
1012    */
1013   @Test
1014   public void testNoHdfsTable() throws Exception {
1015     String table = "NoHdfsTable";
1016     setupTable(table);
1017     assertEquals(ROWKEYS.length, countRows());
1018 
1019     // make sure data in regions, if in hlog only there is no data loss
1020     TEST_UTIL.getHBaseAdmin().flush(table);
1021 
1022     // Mess it up by leaving a giant hole in meta
1023     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes(""),
1024         Bytes.toBytes("A"), false, false, true); // don't rm meta
1025     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1026         Bytes.toBytes("B"), false, false, true); // don't rm meta
1027     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1028         Bytes.toBytes("C"), false, false, true); // don't rm meta
1029     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("C"),
1030         Bytes.toBytes(""), false, false, true); // don't rm meta
1031 
1032     HBaseFsck hbck = doFsck(conf, false);
1033     assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS,
1034         ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS,
1035         ERROR_CODE.NOT_IN_HDFS,});
1036     // holes are separate from overlap groups
1037     assertEquals(0, hbck.getOverlapGroups(table).size());
1038 
1039     // fix hole
1040     doFsck(conf, true); // in 0.92+, meta entries auto create regiondirs
1041 
1042     // check that hole fixed
1043     assertNoErrors(doFsck(conf,false));
1044     assertFalse("Table "+ table + " should have been deleted",
1045         TEST_UTIL.getHBaseAdmin().tableExists(table));
1046   }
1047 
1048   /**
1049    * when the hbase.version file missing, It is fix the fault.
1050    */
1051   @Test
1052   public void testNoVersionFile() throws Exception {
1053     // delete the hbase.version file
1054     Path rootDir = new Path(conf.get(HConstants.HBASE_DIR));
1055     FileSystem fs = rootDir.getFileSystem(conf);
1056     Path versionFile = new Path(rootDir, HConstants.VERSION_FILE_NAME);
1057     fs.delete(versionFile, true);
1058 
1059     // test
1060     HBaseFsck hbck = doFsck(conf, false);
1061     assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_VERSION_FILE });
1062     // fix hbase.version missing
1063     doFsck(conf, true);
1064 
1065     // no version file fixed
1066     assertNoErrors(doFsck(conf, false));
1067   }
1068 
1069   /**
1070    * the region is not deployed when the table is disabled.
1071    */
1072   @Test
1073   public void testRegionShouldNotBeDeployed() throws Exception {
1074     String table = "tableRegionShouldNotBeDeployed";
1075     try {
1076       LOG.info("Starting testRegionShouldNotBeDeployed.");
1077       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
1078       assertTrue(cluster.waitForActiveAndReadyMaster());
1079 
1080       // Create a ZKW to use in the test
1081       ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
1082 
1083       FileSystem filesystem = FileSystem.get(conf);
1084       Path rootdir = filesystem.makeQualified(new Path(conf
1085           .get(HConstants.HBASE_DIR)));
1086 
1087       byte[][] SPLIT_KEYS = new byte[][] { new byte[0], Bytes.toBytes("aaa"),
1088           Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), Bytes.toBytes("ddd") };
1089       HTableDescriptor htdDisabled = new HTableDescriptor(Bytes.toBytes(table));
1090       htdDisabled.addFamily(new HColumnDescriptor(FAM));
1091 
1092       // Write the .tableinfo
1093       FSTableDescriptors
1094           .createTableDescriptor(filesystem, rootdir, htdDisabled);
1095       List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
1096           TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
1097 
1098       // Let's just assign everything to first RS
1099       HRegionServer hrs = cluster.getRegionServer(0);
1100       ServerName serverName = hrs.getServerName();
1101 
1102       // create region files.
1103       TEST_UTIL.getHBaseAdmin().disableTable(table);
1104       TEST_UTIL.getHBaseAdmin().enableTable(table);
1105 
1106       // Region of disable table was opened on RS
1107       TEST_UTIL.getHBaseAdmin().disableTable(table);
1108       HRegionInfo region = disabledRegions.remove(0);
1109       ZKAssign.createNodeOffline(zkw, region, serverName);
1110       hrs.openRegion(region);
1111 
1112       int iTimes = 0;
1113       while (true) {
1114         RegionTransitionData rtd = ZKAssign.getData(zkw,
1115             region.getEncodedName());
1116         if (rtd != null && rtd.getEventType() == EventType.RS_ZK_REGION_OPENED) {
1117           break;
1118         }
1119         Thread.sleep(100);
1120         iTimes++;
1121         if (iTimes >= REGION_ONLINE_TIMEOUT) {
1122           break;
1123         }
1124       }
1125       assertTrue(iTimes < REGION_ONLINE_TIMEOUT);
1126 
1127       HBaseFsck hbck = doFsck(conf, false);
1128       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.SHOULD_NOT_BE_DEPLOYED });
1129 
1130       // fix this fault
1131       doFsck(conf, true);
1132 
1133       // check result
1134       assertNoErrors(doFsck(conf, false));
1135     } finally {
1136       TEST_UTIL.getHBaseAdmin().enableTable(table);
1137       deleteTable(table);
1138     }
1139   }
1140 
1141   /**
1142    * This creates two tables and mess both of them and fix them one by one
1143    */
1144   @Test
1145   public void testFixByTable() throws Exception {
1146     String table1 = "testFixByTable1";
1147     String table2 = "testFixByTable2";
1148     try {
1149       setupTable(table1);
1150       // make sure data in regions, if in hlog only there is no data loss
1151       TEST_UTIL.getHBaseAdmin().flush(table1);
1152       // Mess them up by leaving a hole in the hdfs data
1153       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1154         Bytes.toBytes("C"), false, false, true); // don't rm meta
1155 
1156       setupTable(table2);
1157       // make sure data in regions, if in hlog only there is no data loss
1158       TEST_UTIL.getHBaseAdmin().flush(table2);
1159       // Mess them up by leaving a hole in the hdfs data
1160       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1161         Bytes.toBytes("C"), false, false, true); // don't rm meta
1162 
1163       HBaseFsck hbck = doFsck(conf, false);
1164       assertErrors(hbck, new ERROR_CODE[] {
1165         ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS});
1166 
1167       // fix hole in table 1
1168       doFsck(conf, true, table1);
1169       // check that hole in table 1 fixed
1170       assertNoErrors(doFsck(conf, false, table1));
1171       // check that hole in table 2 still there
1172       assertErrors(doFsck(conf, false, table2),
1173         new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
1174 
1175       // fix hole in table 2
1176       doFsck(conf, true, table2);
1177       // check that hole in both tables fixed
1178       assertNoErrors(doFsck(conf, false));
1179       assertEquals(ROWKEYS.length - 2, countRows());
1180     } finally {
1181       deleteTable(table1);
1182       deleteTable(table2);
1183     }
1184   }
1185   /**
1186    * A split parent in meta, in hdfs, and not deployed
1187    */
1188   @Test
1189   public void testLingeringSplitParent() throws Exception {
1190     String table = "testLingeringSplitParent";
1191     HTable meta = null;
1192     try {
1193       setupTable(table);
1194       assertEquals(ROWKEYS.length, countRows());
1195 
1196       // make sure data in regions, if in hlog only there is no data loss
1197       TEST_UTIL.getHBaseAdmin().flush(table);
1198       HRegionLocation location = tbl.getRegionLocation("B");
1199 
1200       // Delete one region from meta, but not hdfs, unassign it.
1201       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1202         Bytes.toBytes("C"), true, true, false);
1203 
1204       // Create a new meta entry to fake it as a split parent.
1205       meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
1206       HRegionInfo hri = location.getRegionInfo();
1207 
1208       HRegionInfo a = new HRegionInfo(tbl.getTableName(),
1209         Bytes.toBytes("B"), Bytes.toBytes("BM"));
1210       HRegionInfo b = new HRegionInfo(tbl.getTableName(),
1211         Bytes.toBytes("BM"), Bytes.toBytes("C"));
1212       Put p = new Put(hri.getRegionName());
1213       hri.setOffline(true);
1214       hri.setSplit(true);
1215       p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
1216         Writables.getBytes(hri));
1217       p.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
1218         Writables.getBytes(a));
1219       p.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
1220         Writables.getBytes(b));
1221       meta.put(p);
1222       meta.flushCommits();
1223       TEST_UTIL.getHBaseAdmin().flush(HConstants.META_TABLE_NAME);
1224 
1225       HBaseFsck hbck = doFsck(conf, false);
1226       assertErrors(hbck, new ERROR_CODE[] {
1227         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1228 
1229       // regular repair cannot fix lingering split parent
1230       hbck = doFsck(conf, true);
1231       assertErrors(hbck, new ERROR_CODE[] {
1232         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1233       assertFalse(hbck.shouldRerun());
1234       hbck = doFsck(conf, false);
1235       assertErrors(hbck, new ERROR_CODE[] {
1236         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1237 
1238       // fix lingering split parent
1239       hbck = new HBaseFsck(conf);
1240       hbck.connect();
1241       hbck.setDisplayFullReport(); // i.e. -details
1242       hbck.setTimeLag(0);
1243       hbck.setFixSplitParents(true);
1244       hbck.onlineHbck();
1245       assertTrue(hbck.shouldRerun());
1246 
1247       Get get = new Get(hri.getRegionName());
1248       Result result = meta.get(get);
1249       assertTrue(result.getColumn(HConstants.CATALOG_FAMILY,
1250         HConstants.SPLITA_QUALIFIER).isEmpty());
1251       assertTrue(result.getColumn(HConstants.CATALOG_FAMILY,
1252         HConstants.SPLITB_QUALIFIER).isEmpty());
1253       TEST_UTIL.getHBaseAdmin().flush(HConstants.META_TABLE_NAME);
1254 
1255       // fix other issues
1256       doFsck(conf, true);
1257 
1258       // check that all are fixed
1259       assertNoErrors(doFsck(conf, false));
1260       assertEquals(ROWKEYS.length, countRows());
1261     } finally {
1262       deleteTable(table);
1263       IOUtils.closeQuietly(meta);
1264     }
1265   }
1266 
1267   /**
1268    * Tests that LINGERING_SPLIT_PARENT is not erroneously reported for
1269    * valid cases where the daughters are there.
1270    */
1271   @Test
1272   public void testValidLingeringSplitParent() throws Exception {
1273     String table = "testLingeringSplitParent";
1274     HTable meta = null;
1275     try {
1276       setupTable(table);
1277       assertEquals(ROWKEYS.length, countRows());
1278 
1279       // make sure data in regions, if in hlog only there is no data loss
1280       TEST_UTIL.getHBaseAdmin().flush(table);
1281       HRegionLocation location = tbl.getRegionLocation("B");
1282 
1283       meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
1284       HRegionInfo hri = location.getRegionInfo();
1285 
1286       // do a regular split
1287       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1288       byte[] regionName = location.getRegionInfo().getRegionName();
1289       admin.split(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
1290       TestEndToEndSplitTransaction.blockUntilRegionSplit(
1291           TEST_UTIL.getConfiguration(), 60000, regionName, true);
1292 
1293       // TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
1294       // for some time until children references are deleted. HBCK erroneously sees this as
1295       // overlapping regions
1296       HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, null);
1297       assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
1298 
1299       // assert that the split META entry is still there.
1300       Get get = new Get(hri.getRegionName());
1301       Result result = meta.get(get);
1302       assertNotNull(result);
1303       assertNotNull(MetaReader.parseCatalogResult(result).getFirst());
1304 
1305       assertEquals(ROWKEYS.length, countRows());
1306 
1307       // assert that we still have the split regions
1308       assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
1309       assertNoErrors(doFsck(conf, false));
1310     } finally {
1311       deleteTable(table);
1312       IOUtils.closeQuietly(meta);
1313     }
1314   }
1315 
1316   /**
1317    * Split crashed after write to META finished for the parent region, but
1318    * failed to write daughters (pre HBASE-7721 codebase)
1319    */
1320   @Test
1321   public void testSplitDaughtersNotInMeta() throws Exception {
1322     String table = "testSplitdaughtersNotInMeta";
1323     HTable meta = null;
1324     try {
1325       setupTable(table);
1326       assertEquals(ROWKEYS.length, countRows());
1327 
1328       // make sure data in regions, if in hlog only there is no data loss
1329       TEST_UTIL.getHBaseAdmin().flush(table);
1330       HRegionLocation location = tbl.getRegionLocation("B");
1331 
1332       meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
1333       HRegionInfo hri = location.getRegionInfo();
1334 
1335       // do a regular split
1336       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1337       byte[] regionName = location.getRegionInfo().getRegionName();
1338       admin.split(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
1339       TestEndToEndSplitTransaction.blockUntilRegionSplit(
1340           TEST_UTIL.getConfiguration(), 60000, regionName, true);
1341 
1342       PairOfSameType<HRegionInfo> daughters = MetaReader.getDaughterRegions(meta.get(new Get(regionName)));
1343 
1344       // Delete daughter regions from meta, but not hdfs, unassign it.
1345       Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
1346       undeployRegion(admin, hris.get(daughters.getFirst()), daughters.getFirst());
1347       undeployRegion(admin, hris.get(daughters.getSecond()), daughters.getSecond());
1348 
1349       meta.delete(new Delete(daughters.getFirst().getRegionName()));
1350       meta.delete(new Delete(daughters.getSecond().getRegionName()));
1351       meta.flushCommits();
1352 
1353       HBaseFsck hbck = doFsck(conf, false);
1354       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1355           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN}); //no LINGERING_SPLIT_PARENT
1356 
1357       // now fix it. The fix should not revert the region split, but add daughters to META
1358       hbck = doFsck(conf, true, true, false, false, false, false, false, false, null);
1359       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1360           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1361 
1362       // assert that the split META entry is still there.
1363       Get get = new Get(hri.getRegionName());
1364       Result result = meta.get(get);
1365       assertNotNull(result);
1366       assertNotNull(MetaReader.parseCatalogResult(result).getFirst());
1367 
1368       assertEquals(ROWKEYS.length, countRows());
1369 
1370       // assert that we still have the split regions
1371       assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
1372       assertNoErrors(doFsck(conf, false)); //should be fixed by now
1373     } finally {
1374       deleteTable(table);
1375       IOUtils.closeQuietly(meta);
1376     }
1377   }
1378 
1379   /**
1380    * This creates and fixes a bad table with a missing region which is the 1st region -- hole in
1381    * meta and data missing in the fs.
1382    */
1383   @Test
1384   public void testMissingFirstRegion() throws Exception {
1385     String table = "testMissingFirstRegion";
1386     try {
1387       setupTable(table);
1388       assertEquals(ROWKEYS.length, countRows());
1389 
1390       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1391       TEST_UTIL.getHBaseAdmin().disableTable(table);
1392       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes(""), Bytes.toBytes("A"), true,
1393           true, true);
1394       TEST_UTIL.getHBaseAdmin().enableTable(table);
1395 
1396       HBaseFsck hbck = doFsck(conf, false);
1397       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY });
1398       // fix hole
1399       doFsck(conf, true);
1400       // check that hole fixed
1401       assertNoErrors(doFsck(conf, false));
1402     } finally {
1403       deleteTable(table);
1404     }
1405   }
1406   
1407   /**
1408    * This creates and fixes a bad table with missing last region -- hole in meta and data missing in
1409    * the fs.
1410    */
1411   @Test
1412   public void testMissingLastRegion() throws Exception {
1413     String table = "testMissingLastRegion";
1414     try {
1415       setupTable(table);
1416       assertEquals(ROWKEYS.length, countRows());
1417 
1418       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1419       TEST_UTIL.getHBaseAdmin().disableTable(table);
1420       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("C"), Bytes.toBytes(""), true,
1421           true, true);
1422       TEST_UTIL.getHBaseAdmin().enableTable(table);
1423 
1424       HBaseFsck hbck = doFsck(conf, false);
1425       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY });
1426       // fix hole
1427       doFsck(conf, true);
1428       // check that hole fixed
1429       assertNoErrors(doFsck(conf, false));
1430     } finally {
1431       deleteTable(table);
1432     }
1433   }
1434 
1435   /**
1436    * Test -noHdfsChecking option can detect and fix assignments issue.
1437    */
1438   @Test
1439   public void testFixAssignmentsAndNoHdfsChecking() throws Exception {
1440     String table = "testFixAssignmentsAndNoHdfsChecking";
1441     try {
1442       setupTable(table);
1443       assertEquals(ROWKEYS.length, countRows());
1444 
1445       // Mess it up by closing a region
1446       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1447         Bytes.toBytes("B"), true, false, false, false);
1448 
1449       // verify there is no other errors
1450       HBaseFsck hbck = doFsck(conf, false);
1451       assertErrors(hbck, new ERROR_CODE[] {
1452         ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1453 
1454       // verify that noHdfsChecking report the same errors
1455       HBaseFsck fsck = new HBaseFsck(conf);
1456       fsck.connect();
1457       fsck.setDisplayFullReport(); // i.e. -details
1458       fsck.setTimeLag(0);
1459       fsck.setCheckHdfs(false);
1460       fsck.onlineHbck();
1461       assertErrors(fsck, new ERROR_CODE[] {
1462         ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1463 
1464       // verify that fixAssignments works fine with noHdfsChecking
1465       fsck = new HBaseFsck(conf);
1466       fsck.connect();
1467       fsck.setDisplayFullReport(); // i.e. -details
1468       fsck.setTimeLag(0);
1469       fsck.setCheckHdfs(false);
1470       fsck.setFixAssignments(true);
1471       fsck.onlineHbck();
1472       assertTrue(fsck.shouldRerun());
1473       fsck.onlineHbck();
1474       assertNoErrors(fsck);
1475 
1476       assertEquals(ROWKEYS.length, countRows());
1477     } finally {
1478       deleteTable(table);
1479     }
1480   }
1481 
1482   /**
1483    * Test -noHdfsChecking option can detect region is not in meta but deployed.
1484    * However, it can not fix it without checking Hdfs because we need to get
1485    * the region info from Hdfs in this case, then to patch the meta.
1486    */
1487   @Test
1488   public void testFixMetaNotWorkingWithNoHdfsChecking() throws Exception {
1489     String table = "testFixMetaNotWorkingWithNoHdfsChecking";
1490     try {
1491       setupTable(table);
1492       assertEquals(ROWKEYS.length, countRows());
1493 
1494       // Mess it up by deleting a region from the metadata
1495       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1496         Bytes.toBytes("B"), false, true, false, false);
1497 
1498       // verify there is no other errors
1499       HBaseFsck hbck = doFsck(conf, false);
1500       assertErrors(hbck, new ERROR_CODE[] {
1501         ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1502 
1503       // verify that noHdfsChecking report the same errors
1504       HBaseFsck fsck = new HBaseFsck(conf);
1505       fsck.connect();
1506       fsck.setDisplayFullReport(); // i.e. -details
1507       fsck.setTimeLag(0);
1508       fsck.setCheckHdfs(false);
1509       fsck.onlineHbck();
1510       assertErrors(fsck, new ERROR_CODE[] {
1511         ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1512 
1513       // verify that fixMeta doesn't work with noHdfsChecking
1514       fsck = new HBaseFsck(conf);
1515       fsck.connect();
1516       fsck.setDisplayFullReport(); // i.e. -details
1517       fsck.setTimeLag(0);
1518       fsck.setCheckHdfs(false);
1519       fsck.setFixAssignments(true);
1520       fsck.setFixMeta(true);
1521       fsck.onlineHbck();
1522       assertFalse(fsck.shouldRerun());
1523       assertErrors(fsck, new ERROR_CODE[] {
1524         ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1525     } finally {
1526       deleteTable(table);
1527     }
1528   }
1529 
1530   /**
1531    * Test -fixHdfsHoles doesn't work with -noHdfsChecking option,
1532    * and -noHdfsChecking can't detect orphan Hdfs region.
1533    */
1534   @Test
1535   public void testFixHdfsHolesNotWorkingWithNoHdfsChecking() throws Exception {
1536     String table = "testFixHdfsHolesNotWorkingWithNoHdfsChecking";
1537     try {
1538       setupTable(table);
1539       assertEquals(ROWKEYS.length, countRows());
1540 
1541       // Mess it up by creating an overlap in the metadata
1542       TEST_UTIL.getHBaseAdmin().disableTable(table);
1543       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1544         Bytes.toBytes("B"), true, true, false, true);
1545       TEST_UTIL.getHBaseAdmin().enableTable(table);
1546 
1547       HRegionInfo hriOverlap = createRegion(conf, tbl.getTableDescriptor(),
1548         Bytes.toBytes("A2"), Bytes.toBytes("B"));
1549       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
1550       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
1551         .waitForAssignment(hriOverlap);
1552 
1553       HBaseFsck hbck = doFsck(conf, false);
1554       assertErrors(hbck, new ERROR_CODE[] {
1555         ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1556         ERROR_CODE.HOLE_IN_REGION_CHAIN});
1557 
1558       // verify that noHdfsChecking can't detect ORPHAN_HDFS_REGION
1559       HBaseFsck fsck = new HBaseFsck(conf);
1560       fsck.connect();
1561       fsck.setDisplayFullReport(); // i.e. -details
1562       fsck.setTimeLag(0);
1563       fsck.setCheckHdfs(false);
1564       fsck.onlineHbck();
1565       assertErrors(fsck, new ERROR_CODE[] {
1566         ERROR_CODE.HOLE_IN_REGION_CHAIN});
1567 
1568       // verify that fixHdfsHoles doesn't work with noHdfsChecking
1569       fsck = new HBaseFsck(conf);
1570       fsck.connect();
1571       fsck.setDisplayFullReport(); // i.e. -details
1572       fsck.setTimeLag(0);
1573       fsck.setCheckHdfs(false);
1574       fsck.setFixHdfsHoles(true);
1575       fsck.setFixHdfsOverlaps(true);
1576       fsck.setFixHdfsOrphans(true);
1577       fsck.onlineHbck();
1578       assertFalse(fsck.shouldRerun());
1579       assertErrors(fsck, new ERROR_CODE[] {
1580         ERROR_CODE.HOLE_IN_REGION_CHAIN});
1581     } finally {
1582       if (TEST_UTIL.getHBaseAdmin().isTableDisabled(table)) {
1583         TEST_UTIL.getHBaseAdmin().enableTable(table);
1584       }
1585       deleteTable(table);
1586     }
1587   }
1588 
1589   /**
1590    * We don't have an easy way to verify that a flush completed, so we loop until we find a
1591    * legitimate hfile and return it.
1592    * @param fs
1593    * @param table
1594    * @return Path of a flushed hfile.
1595    * @throws IOException
1596    */
1597   Path getFlushedHFile(FileSystem fs, String table) throws IOException {
1598     Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
1599     Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
1600     Path famDir = new Path(regionDir, FAM_STR);
1601 
1602     // keep doing this until we get a legit hfile
1603     while (true) {
1604       FileStatus[] hfFss = fs.listStatus(famDir);
1605       if (hfFss.length == 0) {
1606         continue;
1607       }
1608       for (FileStatus hfs : hfFss) {
1609         if (!hfs.isDir()) {
1610           return hfs.getPath();
1611         }
1612       }
1613     }
1614   }
1615 
1616   /**
1617    * This creates a table and then corrupts an hfile.  Hbck should quarantine the file.
1618    */
1619   @Test(timeout=120000)
1620   public void testQuarantineCorruptHFile() throws Exception {
1621     String table = name.getMethodName();
1622     try {
1623       setupTable(table);
1624       assertEquals(ROWKEYS.length, countRows());
1625       TEST_UTIL.getHBaseAdmin().flush(table); // flush is async.
1626 
1627       FileSystem fs = FileSystem.get(conf);
1628       Path hfile = getFlushedHFile(fs, table);
1629 
1630       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1631       TEST_UTIL.getHBaseAdmin().disableTable(table);
1632 
1633       // create new corrupt file called deadbeef (valid hfile name)
1634       Path corrupt = new Path(hfile.getParent(), "deadbeef");
1635       TestHFile.truncateFile(fs, hfile, corrupt);
1636       LOG.info("Created corrupted file " + corrupt);
1637       HBaseFsck.debugLsr(conf, FSUtils.getRootDir(conf));
1638 
1639       // we cannot enable here because enable never finished due to the corrupt region.
1640       HBaseFsck res = HbckTestingUtil.doHFileQuarantine(conf, table);
1641       assertEquals(res.getRetCode(), 0);
1642       HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
1643       assertEquals(hfcc.getHFilesChecked(), 5);
1644       assertEquals(hfcc.getCorrupted().size(), 1);
1645       assertEquals(hfcc.getFailures().size(), 0);
1646       assertEquals(hfcc.getQuarantined().size(), 1);
1647       assertEquals(hfcc.getMissing().size(), 0);
1648 
1649       // Its been fixed, verify that we can enable.
1650       TEST_UTIL.getHBaseAdmin().enableTable(table);
1651     } finally {
1652       deleteTable(table);
1653     }
1654   }
1655 
1656   /**
1657   * Test that use this should have a timeout, because this method could potentially wait forever.
1658   */
1659   private void doQuarantineTest(String table, HBaseFsck hbck, int check, int corrupt, int fail,
1660       int quar, int missing) throws Exception {
1661     try {
1662       setupTable(table);
1663       assertEquals(ROWKEYS.length, countRows());
1664       TEST_UTIL.getHBaseAdmin().flush(table); // flush is async.
1665 
1666       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1667       TEST_UTIL.getHBaseAdmin().disableTable(table);
1668       
1669       String[] args = {"-sidelineCorruptHFiles", "-repairHoles", "-ignorePreCheckPermission", table};
1670       ExecutorService exec = new ScheduledThreadPoolExecutor(10);
1671       HBaseFsck res = hbck.exec(exec, args);
1672 
1673       HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
1674       assertEquals(hfcc.getHFilesChecked(), check);
1675       assertEquals(hfcc.getCorrupted().size(), corrupt);
1676       assertEquals(hfcc.getFailures().size(), fail);
1677       assertEquals(hfcc.getQuarantined().size(), quar);
1678       assertEquals(hfcc.getMissing().size(), missing);
1679 
1680       // its been fixed, verify that we can enable
1681       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1682       admin.enableTableAsync(table);
1683       while (!admin.isTableEnabled(table)) {
1684         try {
1685           Thread.sleep(250);
1686         } catch (InterruptedException e) {
1687           e.printStackTrace();
1688           fail("Interrupted when trying to enable table " + table);
1689         }
1690       }
1691     } finally {
1692       deleteTable(table);
1693     }
1694   }
1695 
1696   /**
1697    * This creates a table and simulates the race situation where a concurrent compaction or split
1698    * has removed an hfile after the corruption checker learned about it.
1699    */
1700   @Test(timeout=120000)
1701   public void testQuarantineMissingHFile() throws Exception {
1702     String table = name.getMethodName();
1703     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
1704     // inject a fault in the hfcc created.
1705     final FileSystem fs = FileSystem.get(conf);
1706     HBaseFsck hbck = new HBaseFsck(conf, exec) {
1707       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
1708         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
1709           boolean attemptedFirstHFile = false;
1710           protected void checkHFile(Path p) throws IOException {
1711             if (!attemptedFirstHFile) {
1712               attemptedFirstHFile = true;
1713               assertTrue(fs.delete(p, true)); // make sure delete happened.
1714             }
1715             super.checkHFile(p);
1716           }
1717         };
1718       }
1719     };
1720     doQuarantineTest(table, hbck, 4, 0, 0, 0, 1); // 4 attempted, but 1 missing.
1721   }
1722 
1723   /**
1724    * This creates a table and simulates the race situation where a concurrent compaction or split
1725    * has removed an colfam dir before the corruption checker got to it.
1726    */
1727   @Test(timeout=120000)
1728   public void testQuarantineMissingFamdir() throws Exception {
1729     String table = name.getMethodName();
1730     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
1731     // inject a fault in the hfcc created.
1732     final FileSystem fs = FileSystem.get(conf);
1733     HBaseFsck hbck = new HBaseFsck(conf, exec) {
1734       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
1735         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
1736           boolean attemptedFirstFamDir = false;
1737           protected void checkColFamDir(Path p) throws IOException {
1738             if (!attemptedFirstFamDir) {
1739               attemptedFirstFamDir = true;
1740               assertTrue(fs.delete(p, true)); // make sure delete happened.
1741             }
1742             super.checkColFamDir(p);
1743           }
1744         };
1745       }
1746     };
1747     doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
1748   }
1749 
1750   /**
1751    * This creates a table and simulates the race situation where a concurrent compaction or split
1752    * has removed a region dir before the corruption checker got to it.
1753    */
1754   @Test(timeout=120000)
1755   public void testQuarantineMissingRegionDir() throws Exception {
1756     String table = name.getMethodName();
1757     ExecutorService exec = new ScheduledThreadPoolExecutor(10);
1758     // inject a fault in the hfcc created.
1759     final FileSystem fs = FileSystem.get(conf);
1760     HBaseFsck hbck = new HBaseFsck(conf, exec) {
1761       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
1762         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
1763           boolean attemptedFirstRegionDir = false;
1764           protected void checkRegionDir(Path p) throws IOException {
1765             if (!attemptedFirstRegionDir) {
1766               attemptedFirstRegionDir = true;
1767               assertTrue(fs.delete(p, true)); // make sure delete happened.
1768             }
1769             super.checkRegionDir(p);
1770           }
1771         };
1772       }
1773     };
1774     doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
1775   }
1776 
1777   /**
1778    * Test fixing lingering reference file.
1779    */
1780   @Test
1781   public void testLingeringReferenceFile() throws Exception {
1782     String table = "testLingeringReferenceFile";
1783     try {
1784       setupTable(table);
1785       assertEquals(ROWKEYS.length, countRows());
1786 
1787       // Mess it up by creating a fake reference file
1788       FileSystem fs = FileSystem.get(conf);
1789       Path tableDir= FSUtils.getTablePath(FSUtils.getRootDir(conf), table);
1790       Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
1791       Path famDir = new Path(regionDir, FAM_STR);
1792       Path fakeReferenceFile = new Path(famDir, "fbce357483ceea.12144538");
1793       fs.create(fakeReferenceFile);
1794 
1795       HBaseFsck hbck = doFsck(conf, false);
1796       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LINGERING_REFERENCE_HFILE });
1797       // fix reference file
1798       doFsck(conf, true);
1799       // check that reference file fixed
1800       assertNoErrors(doFsck(conf, false));
1801     } finally {
1802       deleteTable(table);
1803     }
1804   }
1805 
1806   /**
1807    * Test pluggable error reporter. It can be plugged in
1808    * from system property or configuration.
1809    */
1810   @Test
1811   public void testErrorReporter() throws Exception {
1812     try {
1813       MockErrorReporter.calledCount = 0;
1814       doFsck(conf, false);
1815       assertEquals(MockErrorReporter.calledCount, 0);
1816 
1817       conf.set("hbasefsck.errorreporter", MockErrorReporter.class.getName());
1818       doFsck(conf, false);
1819       assertTrue(MockErrorReporter.calledCount > 20);
1820     } finally {
1821       conf.set("hbasefsck.errorreporter",
1822         PrintingErrorReporter.class.getName());
1823       MockErrorReporter.calledCount = 0;
1824     }
1825   }
1826 
1827   static class MockErrorReporter implements ErrorReporter {
1828     static int calledCount = 0;
1829 
1830     public void clear() {
1831       calledCount++;
1832     }
1833 
1834     public void report(String message) {
1835       calledCount++;
1836     }
1837 
1838     public void reportError(String message) {
1839       calledCount++;
1840     }
1841 
1842     public void reportError(ERROR_CODE errorCode, String message) {
1843       calledCount++;
1844     }
1845 
1846     public void reportError(ERROR_CODE errorCode, String message, TableInfo table) {
1847       calledCount++;
1848     }
1849 
1850     public void reportError(ERROR_CODE errorCode,
1851         String message, TableInfo table, HbckInfo info) {
1852       calledCount++;
1853     }
1854 
1855     public void reportError(ERROR_CODE errorCode, String message,
1856         TableInfo table, HbckInfo info1, HbckInfo info2) {
1857       calledCount++;
1858     }
1859 
1860     public int summarize() {
1861       return ++calledCount;
1862     }
1863 
1864     public void detail(String details) {
1865       calledCount++;
1866     }
1867 
1868     public ArrayList<ERROR_CODE> getErrorList() {
1869       calledCount++;
1870       return new ArrayList<ERROR_CODE>();
1871     }
1872 
1873     public void progress() {
1874       calledCount++;
1875     }
1876 
1877     public void print(String message) {
1878       calledCount++;
1879     }
1880 
1881     public void resetErrors() {
1882       calledCount++;
1883     }
1884 
1885     public boolean tableHasErrors(TableInfo table) {
1886       calledCount++;
1887       return false;
1888     }
1889   }
1890 
1891   @org.junit.Rule
1892   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1893     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1894 
1895   @org.junit.Rule
1896   public TestName name = new TestName();
1897 }