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