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