View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.util;
20  
21  import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
22  import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
23  import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
24  import static org.junit.Assert.assertEquals;
25  import static org.junit.Assert.assertFalse;
26  import static org.junit.Assert.assertNotEquals;
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.Arrays;
34  import java.util.Collection;
35  import java.util.HashMap;
36  import java.util.HashSet;
37  import java.util.LinkedList;
38  import java.util.List;
39  import java.util.Map;
40  import java.util.Random;
41  import java.util.Map.Entry;
42  import java.util.NavigableMap;
43  import java.util.Set;
44  import java.util.UUID;
45  import java.util.concurrent.Callable;
46  import java.util.concurrent.CountDownLatch;
47  import java.util.concurrent.ExecutorService;
48  import java.util.concurrent.Executors;
49  import java.util.concurrent.Future;
50  import java.util.concurrent.ScheduledThreadPoolExecutor;
51  import java.util.concurrent.SynchronousQueue;
52  import java.util.concurrent.ThreadPoolExecutor;
53  import java.util.concurrent.TimeUnit;
54  import java.util.concurrent.atomic.AtomicBoolean;
55  
56  import org.apache.commons.io.IOUtils;
57  import org.apache.commons.logging.Log;
58  import org.apache.commons.logging.LogFactory;
59  import org.apache.hadoop.conf.Configuration;
60  import org.apache.hadoop.fs.FileStatus;
61  import org.apache.hadoop.fs.FileSystem;
62  import org.apache.hadoop.fs.Path;
63  import org.apache.hadoop.hbase.ClusterStatus;
64  import org.apache.hadoop.hbase.HBaseTestingUtility;
65  import org.apache.hadoop.hbase.HColumnDescriptor;
66  import org.apache.hadoop.hbase.HConstants;
67  import org.apache.hadoop.hbase.HRegionInfo;
68  import org.apache.hadoop.hbase.HRegionLocation;
69  import org.apache.hadoop.hbase.HTableDescriptor;
70  import org.apache.hadoop.hbase.TableExistsException;
71  import org.apache.hadoop.hbase.testclassification.LargeTests;
72  import org.apache.hadoop.hbase.MiniHBaseCluster;
73  import org.apache.hadoop.hbase.RegionLocations;
74  import org.apache.hadoop.hbase.ServerName;
75  import org.apache.hadoop.hbase.TableName;
76  import org.apache.hadoop.hbase.MetaTableAccessor;
77  import org.apache.hadoop.hbase.client.Admin;
78  import org.apache.hadoop.hbase.client.ClusterConnection;
79  import org.apache.hadoop.hbase.client.Connection;
80  import org.apache.hadoop.hbase.client.ConnectionFactory;
81  import org.apache.hadoop.hbase.client.Delete;
82  import org.apache.hadoop.hbase.client.Durability;
83  import org.apache.hadoop.hbase.client.Get;
84  import org.apache.hadoop.hbase.client.HBaseAdmin;
85  import org.apache.hadoop.hbase.client.HConnection;
86  import org.apache.hadoop.hbase.client.HTable;
87  import org.apache.hadoop.hbase.client.MetaScanner;
88  import org.apache.hadoop.hbase.client.Mutation;
89  import org.apache.hadoop.hbase.client.Put;
90  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
91  import org.apache.hadoop.hbase.client.Result;
92  import org.apache.hadoop.hbase.client.ResultScanner;
93  import org.apache.hadoop.hbase.client.RowMutations;
94  import org.apache.hadoop.hbase.client.Scan;
95  import org.apache.hadoop.hbase.client.Table;
96  import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
97  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
98  import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
99  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
100 import org.apache.hadoop.hbase.io.hfile.TestHFile;
101 import org.apache.hadoop.hbase.master.AssignmentManager;
102 import org.apache.hadoop.hbase.master.HMaster;
103 import org.apache.hadoop.hbase.master.RegionState;
104 import org.apache.hadoop.hbase.master.RegionStates;
105 import org.apache.hadoop.hbase.master.TableLockManager;
106 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
107 import org.apache.hadoop.hbase.mob.MobFileName;
108 import org.apache.hadoop.hbase.mob.MobUtils;
109 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
110 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
111 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
112 import org.apache.hadoop.hbase.regionserver.HRegion;
113 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
114 import org.apache.hadoop.hbase.regionserver.HRegionServer;
115 import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
116 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
117 import org.apache.hadoop.hbase.testclassification.LargeTests;
118 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
119 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
120 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
121 import org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter;
122 import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
123 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
124 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
125 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
126 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
127 import org.apache.zookeeper.KeeperException;
128 import org.junit.AfterClass;
129 import org.junit.Assert;
130 import org.junit.Before;
131 import org.junit.BeforeClass;
132 import org.junit.Ignore;
133 import org.junit.Test;
134 import org.junit.experimental.categories.Category;
135 import org.junit.rules.TestName;
136 
137 import com.google.common.collect.Multimap;
138 
139 /**
140  * This tests HBaseFsck's ability to detect reasons for inconsistent tables.
141  */
142 @Category(LargeTests.class)
143 public class TestHBaseFsck {
144   static final int POOL_SIZE = 7;
145 
146   final static Log LOG = LogFactory.getLog(TestHBaseFsck.class);
147   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
148   private final static Configuration conf = TEST_UTIL.getConfiguration();
149   private final static String FAM_STR = "fam";
150   private final static byte[] FAM = Bytes.toBytes(FAM_STR);
151   private final static int REGION_ONLINE_TIMEOUT = 800;
152   private static RegionStates regionStates;
153   private static ExecutorService tableExecutorService;
154   private static ScheduledThreadPoolExecutor hbfsckExecutorService;
155   private static ClusterConnection connection;
156   private static Admin admin;
157 
158   // for the instance, reset every test run
159   private HTable tbl;
160   private final static byte[][] SPLITS = new byte[][] { Bytes.toBytes("A"),
161     Bytes.toBytes("B"), Bytes.toBytes("C") };
162   // one row per region.
163   private final static byte[][] ROWKEYS= new byte[][] {
164     Bytes.toBytes("00"), Bytes.toBytes("50"), Bytes.toBytes("A0"), Bytes.toBytes("A5"),
165     Bytes.toBytes("B0"), Bytes.toBytes("B5"), Bytes.toBytes("C0"), Bytes.toBytes("C5") };
166 
167   @BeforeClass
168   public static void setUpBeforeClass() throws Exception {
169     TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
170       MasterSyncObserver.class.getName());
171 
172     conf.setInt("hbase.regionserver.handler.count", 2);
173     conf.setInt("hbase.regionserver.metahandler.count", 30);
174 
175     conf.setInt("hbase.htable.threads.max", POOL_SIZE);
176     conf.setInt("hbase.hconnection.threads.max", 2 * POOL_SIZE);
177     conf.setInt("hbase.hconnection.threads.core", POOL_SIZE);
178     conf.setInt("hbase.hbck.close.timeout", 2 * REGION_ONLINE_TIMEOUT);
179     conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 8 * REGION_ONLINE_TIMEOUT);
180     TEST_UTIL.startMiniCluster(3);
181 
182     tableExecutorService = new ThreadPoolExecutor(1, POOL_SIZE, 60, TimeUnit.SECONDS,
183         new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("testhbck"));
184 
185     hbfsckExecutorService = new ScheduledThreadPoolExecutor(POOL_SIZE);
186 
187     AssignmentManager assignmentManager =
188       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
189     regionStates = assignmentManager.getRegionStates();
190 
191     connection = (ClusterConnection) TEST_UTIL.getConnection();
192 
193     admin = connection.getAdmin();
194     admin.setBalancerRunning(false, true);
195 
196     TEST_UTIL.waitUntilAllSystemRegionsAssigned();
197 
198   }
199 
200   @AfterClass
201   public static void tearDownAfterClass() throws Exception {
202     tableExecutorService.shutdown();
203     hbfsckExecutorService.shutdown();
204     admin.close();
205     TEST_UTIL.shutdownMiniCluster();
206   }
207 
208   @Before
209   public void setUp() {
210     EnvironmentEdgeManager.reset();
211   }
212 
213   @Test (timeout=180000)
214   public void testHBaseFsck() throws Exception {
215     assertNoErrors(doFsck(conf, false));
216     TableName table = TableName.valueOf("tableBadMetaAssign");
217     HTableDescriptor desc = new HTableDescriptor(table);
218     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
219     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
220     createTable(TEST_UTIL, desc, null);
221 
222     // We created 1 table, should be fine
223     assertNoErrors(doFsck(conf, false));
224 
225     // Now let's mess it up and change the assignment in hbase:meta to
226     // point to a different region server
227     Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
228     Scan scan = new Scan();
229     scan.setStartRow(Bytes.toBytes(table+",,"));
230     ResultScanner scanner = meta.getScanner(scan);
231     HRegionInfo hri = null;
232 
233     Result res = scanner.next();
234     ServerName currServer =
235       ServerName.parseFrom(res.getValue(HConstants.CATALOG_FAMILY,
236           HConstants.SERVER_QUALIFIER));
237     long startCode = Bytes.toLong(res.getValue(HConstants.CATALOG_FAMILY,
238         HConstants.STARTCODE_QUALIFIER));
239 
240     for (JVMClusterUtil.RegionServerThread rs :
241         TEST_UTIL.getHBaseCluster().getRegionServerThreads()) {
242 
243       ServerName sn = rs.getRegionServer().getServerName();
244 
245       // When we find a diff RS, change the assignment and break
246       if (!currServer.getHostAndPort().equals(sn.getHostAndPort()) ||
247           startCode != sn.getStartcode()) {
248         Put put = new Put(res.getRow());
249         put.setDurability(Durability.SKIP_WAL);
250         put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
251           Bytes.toBytes(sn.getHostAndPort()));
252         put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
253           Bytes.toBytes(sn.getStartcode()));
254         meta.put(put);
255         hri = MetaTableAccessor.getHRegionInfo(res);
256         break;
257       }
258     }
259 
260     // Try to fix the data
261     assertErrors(doFsck(conf, true), new ERROR_CODE[]{
262         ERROR_CODE.SERVER_DOES_NOT_MATCH_META});
263 
264     TEST_UTIL.getHBaseCluster().getMaster()
265       .getAssignmentManager().waitForAssignment(hri);
266 
267     // Should be fixed now
268     assertNoErrors(doFsck(conf, false));
269 
270     // comment needed - what is the purpose of this line
271     Table t = connection.getTable(table, tableExecutorService);
272     ResultScanner s = t.getScanner(new Scan());
273     s.close();
274     t.close();
275 
276     scanner.close();
277     meta.close();
278   }
279 
280   @Test(timeout=180000)
281   public void testFixAssignmentsWhenMETAinTransition() throws Exception {
282     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
283     admin.closeRegion(cluster.getServerHoldingMeta(), HRegionInfo.FIRST_META_REGIONINFO);
284     regionStates.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
285     new MetaTableLocator().deleteMetaLocation(cluster.getMaster().getZooKeeper());
286     assertFalse(regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
287     HBaseFsck hbck = doFsck(conf, true);
288     assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.UNKNOWN, ERROR_CODE.NO_META_REGION,
289         ERROR_CODE.NULL_META_REGION });
290     assertNoErrors(doFsck(conf, false));
291   }
292 
293   /**
294    * Create a new region in META.
295    */
296   private HRegionInfo createRegion(final HTableDescriptor
297       htd, byte[] startKey, byte[] endKey)
298       throws IOException {
299     Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
300     HRegionInfo hri = new HRegionInfo(htd.getTableName(), startKey, endKey);
301     MetaTableAccessor.addRegionToMeta(meta, hri);
302     meta.close();
303     return hri;
304   }
305 
306   /**
307    * Debugging method to dump the contents of meta.
308    */
309   private void dumpMeta(TableName tableName) throws IOException {
310     List<byte[]> metaRows = TEST_UTIL.getMetaTableRows(tableName);
311     for (byte[] row : metaRows) {
312       LOG.info(Bytes.toString(row));
313     }
314   }
315 
316   /**
317    * This method is used to undeploy a region -- close it and attempt to
318    * remove its state from the Master.
319    */
320   private void undeployRegion(Connection conn, ServerName sn,
321       HRegionInfo hri) throws IOException, InterruptedException {
322     try {
323       HBaseFsckRepair.closeRegionSilentlyAndWait((HConnection) conn, sn, hri);
324       if (!hri.isMetaTable()) {
325         admin.offline(hri.getRegionName());
326       }
327     } catch (IOException ioe) {
328       LOG.warn("Got exception when attempting to offline region "
329           + Bytes.toString(hri.getRegionName()), ioe);
330     }
331   }
332   /**
333    * Delete a region from assignments, meta, or completely from hdfs.
334    * @param unassign if true unassign region if assigned
335    * @param metaRow  if true remove region's row from META
336    * @param hdfs if true remove region's dir in HDFS
337    */
338   private void deleteRegion(Configuration conf, final HTableDescriptor htd,
339       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
340       boolean hdfs) throws IOException, InterruptedException {
341     deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false, HRegionInfo.DEFAULT_REPLICA_ID);
342   }
343 
344   /**
345    * Delete a region from assignments, meta, or completely from hdfs.
346    * @param unassign if true unassign region if assigned
347    * @param metaRow  if true remove region's row from META
348    * @param hdfs if true remove region's dir in HDFS
349    * @param regionInfoOnly if true remove a region dir's .regioninfo file
350    * @param replicaId replica id
351    */
352   private void deleteRegion(Configuration conf, final HTableDescriptor htd,
353       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
354       boolean hdfs, boolean regionInfoOnly, int replicaId)
355           throws IOException, InterruptedException {
356     LOG.info("** Before delete:");
357     dumpMeta(htd.getTableName());
358 
359     List<HRegionLocation> locations = tbl.getAllRegionLocations();
360     for (HRegionLocation location : locations) {
361       HRegionInfo hri = location.getRegionInfo();
362       ServerName hsa = location.getServerName();
363       if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
364           && Bytes.compareTo(hri.getEndKey(), endKey) == 0
365           && hri.getReplicaId() == replicaId) {
366 
367         LOG.info("RegionName: " +hri.getRegionNameAsString());
368         byte[] deleteRow = hri.getRegionName();
369 
370         if (unassign) {
371           LOG.info("Undeploying region " + hri + " from server " + hsa);
372           undeployRegion(connection, hsa, hri);
373         }
374 
375         if (regionInfoOnly) {
376           LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
377           Path rootDir = FSUtils.getRootDir(conf);
378           FileSystem fs = rootDir.getFileSystem(conf);
379           Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
380               hri.getEncodedName());
381           Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
382           fs.delete(hriPath, true);
383         }
384 
385         if (hdfs) {
386           LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
387           Path rootDir = FSUtils.getRootDir(conf);
388           FileSystem fs = rootDir.getFileSystem(conf);
389           Path p = new Path(FSUtils.getTableDir(rootDir, htd.getTableName()),
390               hri.getEncodedName());
391           HBaseFsck.debugLsr(conf, p);
392           boolean success = fs.delete(p, true);
393           LOG.info("Deleted " + p + " sucessfully? " + success);
394           HBaseFsck.debugLsr(conf, p);
395         }
396 
397         if (metaRow) {
398           try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) {
399             Delete delete = new Delete(deleteRow);
400             meta.delete(delete);
401           }
402         }
403       }
404       LOG.info(hri.toString() + hsa.toString());
405     }
406 
407     TEST_UTIL.getMetaTableRows(htd.getTableName());
408     LOG.info("*** After delete:");
409     dumpMeta(htd.getTableName());
410   }
411 
412   /**
413    * Setup a clean table before we start mucking with it.
414    *
415    * It will set tbl which needs to be closed after test
416    *
417    * @throws IOException
418    * @throws InterruptedException
419    * @throws KeeperException
420    */
421   void setupTable(TableName tablename) throws Exception {
422     setupTableWithRegionReplica(tablename, 1);
423   }
424 
425   /**
426    * Setup a clean table with a certain region_replica count
427    *
428    * It will set tbl which needs to be closed after test
429    *
430    * @param tableName
431    * @param replicaCount
432    * @throws Exception
433    */
434   void setupTableWithRegionReplica(TableName tablename, int replicaCount) throws Exception {
435     HTableDescriptor desc = new HTableDescriptor(tablename);
436     desc.setRegionReplication(replicaCount);
437     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
438     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
439     createTable(TEST_UTIL, desc, SPLITS);
440 
441     tbl = (HTable) connection.getTable(tablename, tableExecutorService);
442     List<Put> puts = new ArrayList<Put>();
443     for (byte[] row : ROWKEYS) {
444       Put p = new Put(row);
445       p.add(FAM, Bytes.toBytes("val"), row);
446       puts.add(p);
447     }
448     tbl.put(puts);
449     tbl.flushCommits();
450   }
451 
452   /**
453    * Setup a clean table with a mob-enabled column.
454    *
455    * @param tableName The name of a table to be created.
456    * @throws Exception
457    */
458   void setupMobTable(TableName tablename) throws Exception {
459     HTableDescriptor desc = new HTableDescriptor(tablename);
460     HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
461     hcd.setMobEnabled(true);
462     hcd.setMobThreshold(0);
463     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
464     createTable(TEST_UTIL, desc, SPLITS);
465 
466     tbl = (HTable) connection.getTable(tablename, tableExecutorService);
467     List<Put> puts = new ArrayList<Put>();
468     for (byte[] row : ROWKEYS) {
469       Put p = new Put(row);
470       p.add(FAM, Bytes.toBytes("val"), row);
471       puts.add(p);
472     }
473     tbl.put(puts);
474     tbl.flushCommits();
475   }
476 
477   /**
478    * Counts the number of rows to verify data loss or non-dataloss.
479    */
480   int countRows() throws IOException {
481      Scan s = new Scan();
482      ResultScanner rs = tbl.getScanner(s);
483      int i = 0;
484      while(rs.next() !=null) {
485        i++;
486      }
487      return i;
488   }
489 
490   /**
491    * Counts the number of rows to verify data loss or non-dataloss.
492    */
493   int countRows(byte[] start, byte[] end) throws IOException {
494     Scan s = new Scan(start, end);
495     ResultScanner rs = tbl.getScanner(s);
496     int i = 0;
497     while (rs.next() != null) {
498       i++;
499     }
500     return i;
501   }
502 
503   /**
504    * delete table in preparation for next test
505    *
506    * @param tablename
507    * @throws IOException
508    */
509   void cleanupTable(TableName tablename) throws Exception {
510     if (tbl != null) {
511       tbl.close();
512       tbl = null;
513     }
514 
515     ((ClusterConnection) connection).clearRegionCache();
516     deleteTable(TEST_UTIL, tablename);
517   }
518 
519   /**
520    * This creates a clean table and confirms that the table is clean.
521    */
522   @Test (timeout=180000)
523   public void testHBaseFsckClean() throws Exception {
524     assertNoErrors(doFsck(conf, false));
525     TableName table = TableName.valueOf("tableClean");
526     try {
527       HBaseFsck hbck = doFsck(conf, false);
528       assertNoErrors(hbck);
529 
530       setupTable(table);
531       assertEquals(ROWKEYS.length, countRows());
532 
533       // We created 1 table, should be fine
534       hbck = doFsck(conf, false);
535       assertNoErrors(hbck);
536       assertEquals(0, hbck.getOverlapGroups(table).size());
537       assertEquals(ROWKEYS.length, countRows());
538     } finally {
539       cleanupTable(table);
540     }
541   }
542 
543   /**
544    * Test thread pooling in the case where there are more regions than threads
545    */
546   @Test (timeout=180000)
547   public void testHbckThreadpooling() throws Exception {
548     TableName table =
549         TableName.valueOf("tableDupeStartKey");
550     try {
551       // Create table with 4 regions
552       setupTable(table);
553 
554       // limit number of threads to 1.
555       Configuration newconf = new Configuration(conf);
556       newconf.setInt("hbasefsck.numthreads", 1);
557       assertNoErrors(doFsck(newconf, false));
558 
559       // We should pass without triggering a RejectedExecutionException
560     } finally {
561       cleanupTable(table);
562     }
563   }
564 
565   @Test (timeout=180000)
566   public void testHbckFixOrphanTable() throws Exception {
567     TableName table = TableName.valueOf("tableInfo");
568     FileSystem fs = null;
569     Path tableinfo = null;
570     try {
571       setupTable(table);
572 
573       Path hbaseTableDir = FSUtils.getTableDir(
574           FSUtils.getRootDir(conf), table);
575       fs = hbaseTableDir.getFileSystem(conf);
576       FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
577       tableinfo = status.getPath();
578       fs.rename(tableinfo, new Path("/.tableinfo"));
579 
580       //to report error if .tableinfo is missing.
581       HBaseFsck hbck = doFsck(conf, false);
582       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_TABLEINFO_FILE });
583 
584       // fix OrphanTable with default .tableinfo (htd not yet cached on master)
585       hbck = doFsck(conf, true);
586       assertNoErrors(hbck);
587       status = null;
588       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
589       assertNotNull(status);
590 
591       HTableDescriptor htd = admin.getTableDescriptor(table);
592       htd.setValue("NOT_DEFAULT", "true");
593       admin.disableTable(table);
594       admin.modifyTable(table, htd);
595       admin.enableTable(table);
596       fs.delete(status.getPath(), true);
597 
598       // fix OrphanTable with cache
599       htd = admin.getTableDescriptor(table); // warms up cached htd on master
600       hbck = doFsck(conf, true);
601       assertNoErrors(hbck);
602       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
603       assertNotNull(status);
604       htd = admin.getTableDescriptor(table);
605       assertEquals(htd.getValue("NOT_DEFAULT"), "true");
606     } finally {
607       fs.rename(new Path("/.tableinfo"), tableinfo);
608       cleanupTable(table);
609     }
610   }
611 
612   /**
613    * This test makes sure that parallel instances of Hbck is disabled.
614    *
615    * @throws Exception
616    */
617   @Test (timeout=180000)
618   public void testParallelHbck() throws Exception {
619     final ExecutorService service;
620     final Future<HBaseFsck> hbck1,hbck2;
621 
622     class RunHbck implements Callable<HBaseFsck>{
623       boolean fail = true;
624       @Override
625       public HBaseFsck call(){
626         Configuration c = new Configuration(conf);
627         c.setInt("hbase.hbck.lockfile.attempts", 1);
628         // HBASE-13574 found that in HADOOP-2.6 and later, the create file would internally retry.
629         // To avoid flakiness of the test, set low max wait time.
630         c.setInt("hbase.hbck.lockfile.maxwaittime", 3);
631         try{
632           return doFsck(c, false);
633         } catch(Exception e){
634           if (e.getMessage().contains("Duplicate hbck")) {
635             fail = false;
636           }
637         }
638         // If we reach here, then an exception was caught
639         if (fail) fail();
640         return null;
641       }
642     }
643     service = Executors.newFixedThreadPool(2);
644     hbck1 = service.submit(new RunHbck());
645     hbck2 = service.submit(new RunHbck());
646     service.shutdown();
647     //wait for 15 seconds, for both hbck calls finish
648     service.awaitTermination(15, TimeUnit.SECONDS);
649     HBaseFsck h1 = hbck1.get();
650     HBaseFsck h2 = hbck2.get();
651     // Make sure only one of the calls was successful
652     assert(h1 == null || h2 == null);
653     if (h1 != null) {
654       assert(h1.getRetCode() >= 0);
655     }
656     if (h2 != null) {
657       assert(h2.getRetCode() >= 0);
658     }
659   }
660 
661   /**
662    * This test makes sure that with enough retries both parallel instances
663    * of hbck will be completed successfully.
664    *
665    * @throws Exception
666    */
667   @Test (timeout=180000)
668   public void testParallelWithRetriesHbck() throws Exception {
669     final ExecutorService service;
670     final Future<HBaseFsck> hbck1,hbck2;
671 
672     // With the ExponentialBackoffPolicyWithLimit (starting with 200 milliseconds sleep time, and
673     // max sleep time of 5 seconds), we can retry around 15 times within 80 seconds before bail out.
674     //
675     // Note: the reason to use 80 seconds is that in HADOOP-2.6 and later, the create file would
676     // retry up to HdfsConstants.LEASE_SOFTLIMIT_PERIOD (60 seconds).  See HBASE-13574 for more
677     // details.
678     final int timeoutInSeconds = 80;
679     final int sleepIntervalInMilliseconds = 200;
680     final int maxSleepTimeInMilliseconds = 6000;
681     final int maxRetryAttempts = 15;
682 
683     class RunHbck implements Callable<HBaseFsck>{
684 
685       @Override
686       public HBaseFsck call() throws Exception {
687         // Increase retry attempts to make sure the non-active hbck doesn't get starved
688         Configuration c = new Configuration(conf);
689         c.setInt("hbase.hbck.lockfile.maxwaittime", timeoutInSeconds);
690         c.setInt("hbase.hbck.lockfile.attempt.sleep.interval", sleepIntervalInMilliseconds);
691         c.setInt("hbase.hbck.lockfile.attempt.maxsleeptime", maxSleepTimeInMilliseconds);
692         c.setInt("hbase.hbck.lockfile.attempts", maxRetryAttempts);
693         return doFsck(c, false);
694       }
695     }
696 
697     service = Executors.newFixedThreadPool(2);
698     hbck1 = service.submit(new RunHbck());
699     hbck2 = service.submit(new RunHbck());
700     service.shutdown();
701     //wait for some time, for both hbck calls finish
702     service.awaitTermination(timeoutInSeconds * 2, TimeUnit.SECONDS);
703     HBaseFsck h1 = hbck1.get();
704     HBaseFsck h2 = hbck2.get();
705     // Both should be successful
706     assertNotNull(h1);
707     assertNotNull(h2);
708     assert(h1.getRetCode() >= 0);
709     assert(h2.getRetCode() >= 0);
710 
711   }
712 
713   /**
714    * This create and fixes a bad table with regions that have a duplicate
715    * start key
716    */
717   @Test (timeout=180000)
718   public void testDupeStartKey() throws Exception {
719     TableName table =
720         TableName.valueOf("tableDupeStartKey");
721     try {
722       setupTable(table);
723       assertNoErrors(doFsck(conf, false));
724       assertEquals(ROWKEYS.length, countRows());
725 
726       // Now let's mess it up, by adding a region with a duplicate startkey
727       HRegionInfo hriDupe =
728           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("A2"));
729       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
730       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
731           .waitForAssignment(hriDupe);
732       ServerName server = regionStates.getRegionServerOfRegion(hriDupe);
733       TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
734 
735       HBaseFsck hbck = doFsck(conf, false);
736       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
737             ERROR_CODE.DUPE_STARTKEYS});
738       assertEquals(2, hbck.getOverlapGroups(table).size());
739       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
740 
741       // fix the degenerate region.
742       doFsck(conf,true);
743 
744       // check that the degenerate region is gone and no data loss
745       HBaseFsck hbck2 = doFsck(conf,false);
746       assertNoErrors(hbck2);
747       assertEquals(0, hbck2.getOverlapGroups(table).size());
748       assertEquals(ROWKEYS.length, countRows());
749     } finally {
750       cleanupTable(table);
751     }
752   }
753 
754   /*
755    * This creates a table with region_replica > 1 and verifies hbck runs
756    * successfully
757    */
758   @Test (timeout=180000)
759   public void testHbckWithRegionReplica() throws Exception {
760     TableName table =
761         TableName.valueOf("testHbckWithRegionReplica");
762     try {
763       setupTableWithRegionReplica(table, 2);
764       TEST_UTIL.getHBaseAdmin().flush(table.getName());
765       assertNoErrors(doFsck(conf, false));
766     } finally {
767       cleanupTable(table);
768     }
769   }
770 
771   @Test
772   public void testHbckWithFewerReplica() throws Exception {
773     TableName table =
774         TableName.valueOf("testHbckWithFewerReplica");
775     try {
776       setupTableWithRegionReplica(table, 2);
777       TEST_UTIL.getHBaseAdmin().flush(table.getName());
778       assertNoErrors(doFsck(conf, false));
779       assertEquals(ROWKEYS.length, countRows());
780       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
781           Bytes.toBytes("C"), true, false, false, false, 1); // unassign one replica
782       // check that problem exists
783       HBaseFsck hbck = doFsck(conf, false);
784       assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.NOT_DEPLOYED});
785       // fix the problem
786       hbck = doFsck(conf, true);
787       // run hbck again to make sure we don't see any errors
788       hbck = doFsck(conf, false);
789       assertErrors(hbck, new ERROR_CODE[]{});
790     } finally {
791       cleanupTable(table);
792     }
793   }
794 
795   @Test
796   public void testHbckWithExcessReplica() throws Exception {
797     TableName table =
798         TableName.valueOf("testHbckWithExcessReplica");
799     try {
800       setupTableWithRegionReplica(table, 2);
801       TEST_UTIL.getHBaseAdmin().flush(table.getName());
802       assertNoErrors(doFsck(conf, false));
803       assertEquals(ROWKEYS.length, countRows());
804       // the next few lines inject a location in meta for a replica, and then
805       // asks the master to assign the replica (the meta needs to be injected
806       // for the master to treat the request for assignment as valid; the master
807       // checks the region is valid either from its memory or meta)
808       HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
809       List<HRegionInfo> regions = TEST_UTIL.getHBaseAdmin().getTableRegions(table);
810       byte[] startKey = Bytes.toBytes("B");
811       byte[] endKey = Bytes.toBytes("C");
812       byte[] metaKey = null;
813       HRegionInfo newHri = null;
814       for (HRegionInfo h : regions) {
815         if (Bytes.compareTo(h.getStartKey(), startKey) == 0  &&
816             Bytes.compareTo(h.getEndKey(), endKey) == 0 &&
817             h.getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
818           metaKey = h.getRegionName();
819           //create a hri with replicaId as 2 (since we already have replicas with replicaid 0 and 1)
820           newHri = RegionReplicaUtil.getRegionInfoForReplica(h, 2);
821           break;
822         }
823       }
824       Put put = new Put(metaKey);
825       ServerName sn = TEST_UTIL.getHBaseAdmin().getClusterStatus().getServers()
826           .toArray(new ServerName[0])[0];
827       //add a location with replicaId as 2 (since we already have replicas with replicaid 0 and 1)
828       MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, 2);
829       meta.put(put);
830       meta.flushCommits();
831       // assign the new replica
832       HBaseFsckRepair.fixUnassigned((HBaseAdmin)TEST_UTIL.getHBaseAdmin(), newHri);
833       HBaseFsckRepair.waitUntilAssigned((HBaseAdmin)TEST_UTIL.getHBaseAdmin(), newHri);
834       // now reset the meta row to its original value
835       Delete delete = new Delete(metaKey);
836       delete.deleteColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(2));
837       delete.deleteColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(2));
838       delete.deleteColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(2));
839       meta.delete(delete);
840       meta.flushCommits();
841       meta.close();
842       // check that problem exists
843       HBaseFsck hbck = doFsck(conf, false);
844       assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.NOT_IN_META});
845       // fix the problem
846       hbck = doFsck(conf, true);
847       // run hbck again to make sure we don't see any errors
848       hbck = doFsck(conf, false);
849       assertErrors(hbck, new ERROR_CODE[]{});
850     } finally {
851       cleanupTable(table);
852     }
853   }
854   /**
855    * Get region info from local cluster.
856    */
857   Map<ServerName, List<String>> getDeployedHRIs(final HBaseAdmin admin) throws IOException {
858     ClusterStatus status = admin.getClusterStatus();
859     Collection<ServerName> regionServers = status.getServers();
860     Map<ServerName, List<String>> mm =
861         new HashMap<ServerName, List<String>>();
862     for (ServerName hsi : regionServers) {
863       AdminProtos.AdminService.BlockingInterface server = ((HConnection) connection).getAdmin(hsi);
864 
865       // list all online regions from this region server
866       List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);
867       List<String> regionNames = new ArrayList<String>();
868       for (HRegionInfo hri : regions) {
869         regionNames.add(hri.getRegionNameAsString());
870       }
871       mm.put(hsi, regionNames);
872     }
873     return mm;
874   }
875 
876   /**
877    * Returns the HSI a region info is on.
878    */
879   ServerName findDeployedHSI(Map<ServerName, List<String>> mm, HRegionInfo hri) {
880     for (Map.Entry<ServerName,List <String>> e : mm.entrySet()) {
881       if (e.getValue().contains(hri.getRegionNameAsString())) {
882         return e.getKey();
883       }
884     }
885     return null;
886   }
887 
888   /**
889    * This create and fixes a bad table with regions that have a duplicate
890    * start key
891    */
892   @Test (timeout=180000)
893   public void testDupeRegion() throws Exception {
894     TableName table =
895         TableName.valueOf("tableDupeRegion");
896     try {
897       setupTable(table);
898       assertNoErrors(doFsck(conf, false));
899       assertEquals(ROWKEYS.length, countRows());
900 
901       // Now let's mess it up, by adding a region with a duplicate startkey
902       HRegionInfo hriDupe =
903           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"));
904 
905       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
906       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
907           .waitForAssignment(hriDupe);
908       ServerName server = regionStates.getRegionServerOfRegion(hriDupe);
909       TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
910 
911       // Yikes! The assignment manager can't tell between diff between two
912       // different regions with the same start/endkeys since it doesn't
913       // differentiate on ts/regionId!  We actually need to recheck
914       // deployments!
915       while (findDeployedHSI(getDeployedHRIs((HBaseAdmin) admin), hriDupe) == null) {
916         Thread.sleep(250);
917       }
918 
919       LOG.debug("Finished assignment of dupe region");
920 
921       // TODO why is dupe region different from dupe start keys?
922       HBaseFsck hbck = doFsck(conf, false);
923       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
924             ERROR_CODE.DUPE_STARTKEYS});
925       assertEquals(2, hbck.getOverlapGroups(table).size());
926       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
927 
928       // fix the degenerate region.
929       doFsck(conf,true);
930 
931       // check that the degenerate region is gone and no data loss
932       HBaseFsck hbck2 = doFsck(conf,false);
933       assertNoErrors(hbck2);
934       assertEquals(0, hbck2.getOverlapGroups(table).size());
935       assertEquals(ROWKEYS.length, countRows());
936     } finally {
937       cleanupTable(table);
938     }
939   }
940 
941   /**
942    * This creates and fixes a bad table with regions that has startkey == endkey
943    */
944   @Test (timeout=180000)
945   public void testDegenerateRegions() throws Exception {
946     TableName table = TableName.valueOf("tableDegenerateRegions");
947     try {
948       setupTable(table);
949       assertNoErrors(doFsck(conf,false));
950       assertEquals(ROWKEYS.length, countRows());
951 
952       // Now let's mess it up, by adding a region with a duplicate startkey
953       HRegionInfo hriDupe =
954           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("B"));
955       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriDupe);
956       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
957           .waitForAssignment(hriDupe);
958       ServerName server = regionStates.getRegionServerOfRegion(hriDupe);
959       TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
960 
961       HBaseFsck hbck = doFsck(conf,false);
962       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DEGENERATE_REGION, ERROR_CODE.DUPE_STARTKEYS,
963           ERROR_CODE.DUPE_STARTKEYS });
964       assertEquals(2, hbck.getOverlapGroups(table).size());
965       assertEquals(ROWKEYS.length, countRows());
966 
967       // fix the degenerate region.
968       doFsck(conf,true);
969 
970       // check that the degenerate region is gone and no data loss
971       HBaseFsck hbck2 = doFsck(conf,false);
972       assertNoErrors(hbck2);
973       assertEquals(0, hbck2.getOverlapGroups(table).size());
974       assertEquals(ROWKEYS.length, countRows());
975     } finally {
976       cleanupTable(table);
977     }
978   }
979 
980   /**
981    * This creates and fixes a bad table where a region is completely contained
982    * by another region.
983    */
984   @Test (timeout=180000)
985   public void testContainedRegionOverlap() throws Exception {
986     TableName table =
987         TableName.valueOf("tableContainedRegionOverlap");
988     try {
989       setupTable(table);
990       assertEquals(ROWKEYS.length, countRows());
991 
992       // Mess it up by creating an overlap in the metadata
993       HRegionInfo hriOverlap =
994           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
995       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
996       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
997           .waitForAssignment(hriOverlap);
998       ServerName server = regionStates.getRegionServerOfRegion(hriOverlap);
999       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
1000 
1001       HBaseFsck hbck = doFsck(conf, false);
1002       assertErrors(hbck, new ERROR_CODE[] {
1003           ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
1004       assertEquals(2, hbck.getOverlapGroups(table).size());
1005       assertEquals(ROWKEYS.length, countRows());
1006 
1007       // fix the problem.
1008       doFsck(conf, true);
1009 
1010       // verify that overlaps are fixed
1011       HBaseFsck hbck2 = doFsck(conf,false);
1012       assertNoErrors(hbck2);
1013       assertEquals(0, hbck2.getOverlapGroups(table).size());
1014       assertEquals(ROWKEYS.length, countRows());
1015     } finally {
1016       cleanupTable(table);
1017     }
1018   }
1019 
1020   /**
1021    * This creates and fixes a bad table where an overlap group of
1022    * 3 regions. Set HBaseFsck.maxMerge to 2 to trigger sideline overlapped
1023    * region. Mess around the meta data so that closeRegion/offlineRegion
1024    * throws exceptions.
1025    */
1026   @Test (timeout=180000)
1027   public void testSidelineOverlapRegion() throws Exception {
1028     TableName table =
1029         TableName.valueOf("testSidelineOverlapRegion");
1030     try {
1031       setupTable(table);
1032       assertEquals(ROWKEYS.length, countRows());
1033 
1034       // Mess it up by creating an overlap
1035       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
1036       HMaster master = cluster.getMaster();
1037       HRegionInfo hriOverlap1 =
1038           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("AB"));
1039       master.assignRegion(hriOverlap1);
1040       master.getAssignmentManager().waitForAssignment(hriOverlap1);
1041       HRegionInfo hriOverlap2 =
1042           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("AB"), Bytes.toBytes("B"));
1043       master.assignRegion(hriOverlap2);
1044       master.getAssignmentManager().waitForAssignment(hriOverlap2);
1045 
1046       HBaseFsck hbck = doFsck(conf, false);
1047       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.DUPE_STARTKEYS,
1048         ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.OVERLAP_IN_REGION_CHAIN});
1049       assertEquals(3, hbck.getOverlapGroups(table).size());
1050       assertEquals(ROWKEYS.length, countRows());
1051 
1052       // mess around the overlapped regions, to trigger NotServingRegionException
1053       Multimap<byte[], HbckInfo> overlapGroups = hbck.getOverlapGroups(table);
1054       ServerName serverName = null;
1055       byte[] regionName = null;
1056       for (HbckInfo hbi: overlapGroups.values()) {
1057         if ("A".equals(Bytes.toString(hbi.getStartKey()))
1058             && "B".equals(Bytes.toString(hbi.getEndKey()))) {
1059           regionName = hbi.getRegionName();
1060 
1061           // get an RS not serving the region to force bad assignment info in to META.
1062           int k = cluster.getServerWith(regionName);
1063           for (int i = 0; i < 3; i++) {
1064             if (i != k) {
1065               HRegionServer rs = cluster.getRegionServer(i);
1066               serverName = rs.getServerName();
1067               break;
1068             }
1069           }
1070 
1071           HBaseFsckRepair.closeRegionSilentlyAndWait((HConnection) connection,
1072               cluster.getRegionServer(k).getServerName(), hbi.getHdfsHRI());
1073           admin.offline(regionName);
1074           break;
1075         }
1076       }
1077 
1078       assertNotNull(regionName);
1079       assertNotNull(serverName);
1080       try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) {
1081         Put put = new Put(regionName);
1082         put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
1083             Bytes.toBytes(serverName.getHostAndPort()));
1084         meta.put(put);
1085       }
1086 
1087       // fix the problem.
1088       HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
1089       fsck.connect();
1090       fsck.setDisplayFullReport(); // i.e. -details
1091       fsck.setTimeLag(0);
1092       fsck.setFixAssignments(true);
1093       fsck.setFixMeta(true);
1094       fsck.setFixHdfsHoles(true);
1095       fsck.setFixHdfsOverlaps(true);
1096       fsck.setFixHdfsOrphans(true);
1097       fsck.setFixVersionFile(true);
1098       fsck.setSidelineBigOverlaps(true);
1099       fsck.setMaxMerge(2);
1100       fsck.onlineHbck();
1101       fsck.close();
1102 
1103       // verify that overlaps are fixed, and there are less rows
1104       // since one region is sidelined.
1105       HBaseFsck hbck2 = doFsck(conf,false);
1106       assertNoErrors(hbck2);
1107       assertEquals(0, hbck2.getOverlapGroups(table).size());
1108       assertTrue(ROWKEYS.length > countRows());
1109     } finally {
1110       cleanupTable(table);
1111     }
1112   }
1113 
1114   /**
1115    * This creates and fixes a bad table where a region is completely contained
1116    * by another region, and there is a hole (sort of like a bad split)
1117    */
1118   @Test (timeout=180000)
1119   public void testOverlapAndOrphan() throws Exception {
1120     TableName table =
1121         TableName.valueOf("tableOverlapAndOrphan");
1122     try {
1123       setupTable(table);
1124       assertEquals(ROWKEYS.length, countRows());
1125 
1126       // Mess it up by creating an overlap in the metadata
1127       admin.disableTable(table);
1128       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1129           Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
1130       TEST_UTIL.getHBaseAdmin().enableTable(table);
1131 
1132       HRegionInfo hriOverlap =
1133           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
1134       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
1135       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
1136           .waitForAssignment(hriOverlap);
1137       ServerName server = regionStates.getRegionServerOfRegion(hriOverlap);
1138       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
1139 
1140       HBaseFsck hbck = doFsck(conf, false);
1141       assertErrors(hbck, new ERROR_CODE[] {
1142           ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1143           ERROR_CODE.HOLE_IN_REGION_CHAIN});
1144 
1145       // fix the problem.
1146       doFsck(conf, true);
1147 
1148       // verify that overlaps are fixed
1149       HBaseFsck hbck2 = doFsck(conf,false);
1150       assertNoErrors(hbck2);
1151       assertEquals(0, hbck2.getOverlapGroups(table).size());
1152       assertEquals(ROWKEYS.length, countRows());
1153     } finally {
1154       cleanupTable(table);
1155     }
1156   }
1157 
1158   /**
1159    * This creates and fixes a bad table where a region overlaps two regions --
1160    * a start key contained in another region and its end key is contained in
1161    * yet another region.
1162    */
1163   @Test (timeout=180000)
1164   public void testCoveredStartKey() throws Exception {
1165     TableName table =
1166         TableName.valueOf("tableCoveredStartKey");
1167     try {
1168       setupTable(table);
1169       assertEquals(ROWKEYS.length, countRows());
1170 
1171       // Mess it up by creating an overlap in the metadata
1172       HRegionInfo hriOverlap =
1173           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B2"));
1174       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
1175       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
1176           .waitForAssignment(hriOverlap);
1177       ServerName server = regionStates.getRegionServerOfRegion(hriOverlap);
1178       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
1179 
1180       HBaseFsck hbck = doFsck(conf, false);
1181       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.OVERLAP_IN_REGION_CHAIN,
1182           ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
1183       assertEquals(3, hbck.getOverlapGroups(table).size());
1184       assertEquals(ROWKEYS.length, countRows());
1185 
1186       // fix the problem.
1187       doFsck(conf, true);
1188 
1189       // verify that overlaps are fixed
1190       HBaseFsck hbck2 = doFsck(conf, false);
1191       assertErrors(hbck2, new ERROR_CODE[0]);
1192       assertEquals(0, hbck2.getOverlapGroups(table).size());
1193       assertEquals(ROWKEYS.length, countRows());
1194     } finally {
1195       cleanupTable(table);
1196     }
1197   }
1198 
1199   /**
1200    * This creates and fixes a bad table with a missing region -- hole in meta
1201    * and data missing in the fs.
1202    */
1203   @Test (timeout=180000)
1204   public void testRegionHole() throws Exception {
1205     TableName table =
1206         TableName.valueOf("tableRegionHole");
1207     try {
1208       setupTable(table);
1209       assertEquals(ROWKEYS.length, countRows());
1210 
1211       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1212       admin.disableTable(table);
1213       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1214           Bytes.toBytes("C"), true, true, true);
1215       admin.enableTable(table);
1216 
1217       HBaseFsck hbck = doFsck(conf, false);
1218       assertErrors(hbck, new ERROR_CODE[] {
1219           ERROR_CODE.HOLE_IN_REGION_CHAIN});
1220       // holes are separate from overlap groups
1221       assertEquals(0, hbck.getOverlapGroups(table).size());
1222 
1223       // fix hole
1224       doFsck(conf, true);
1225 
1226       // check that hole fixed
1227       assertNoErrors(doFsck(conf,false));
1228       assertEquals(ROWKEYS.length - 2 , countRows()); // lost a region so lost a row
1229     } finally {
1230       cleanupTable(table);
1231     }
1232   }
1233 
1234   /**
1235    * This creates and fixes a bad table with a missing region -- hole in meta
1236    * and data present but .regioinfino missing (an orphan hdfs region)in the fs.
1237    */
1238   @Test (timeout=180000)
1239   public void testHDFSRegioninfoMissing() throws Exception {
1240     TableName table = TableName.valueOf("tableHDFSRegioninfoMissing");
1241     try {
1242       setupTable(table);
1243       assertEquals(ROWKEYS.length, countRows());
1244 
1245       // Mess it up by leaving a hole in the meta data
1246       admin.disableTable(table);
1247       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1248           Bytes.toBytes("C"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
1249       TEST_UTIL.getHBaseAdmin().enableTable(table);
1250 
1251       HBaseFsck hbck = doFsck(conf, false);
1252       assertErrors(hbck, new ERROR_CODE[] {
1253           ERROR_CODE.ORPHAN_HDFS_REGION,
1254           ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1255           ERROR_CODE.HOLE_IN_REGION_CHAIN});
1256       // holes are separate from overlap groups
1257       assertEquals(0, hbck.getOverlapGroups(table).size());
1258 
1259       // fix hole
1260       doFsck(conf, true);
1261 
1262       // check that hole fixed
1263       assertNoErrors(doFsck(conf, false));
1264       assertEquals(ROWKEYS.length, countRows());
1265     } finally {
1266       cleanupTable(table);
1267     }
1268   }
1269 
1270   /**
1271    * This creates and fixes a bad table with a missing region -- hole in meta and data present but
1272    * .regioninfo missing (an orphan hdfs region)in the fs. At last we check every row was present
1273    * at the correct region.
1274    */
1275   @Test(timeout = 180000)
1276   public void testHDFSRegioninfoMissingAndCheckRegionBoundary() throws Exception {
1277     TableName table = TableName.valueOf("testHDFSRegioninfoMissingAndCheckRegionBoundary");
1278     try {
1279       setupTable(table);
1280       assertEquals(ROWKEYS.length, countRows());
1281 
1282       // Mess it up by leaving a hole in the meta data
1283       admin.disableTable(table);
1284       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
1285         true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
1286       admin.enableTable(table);
1287 
1288       HBaseFsck hbck = doFsck(conf, false);
1289       assertErrors(hbck,
1290         new HBaseFsck.ErrorReporter.ERROR_CODE[] {
1291             HBaseFsck.ErrorReporter.ERROR_CODE.ORPHAN_HDFS_REGION,
1292             HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1293             HBaseFsck.ErrorReporter.ERROR_CODE.HOLE_IN_REGION_CHAIN });
1294       // holes are separate from overlap groups
1295       assertEquals(0, hbck.getOverlapGroups(table).size());
1296 
1297       // fix hole
1298       doFsck(conf, true);
1299 
1300       // check that hole fixed
1301       assertNoErrors(doFsck(conf, false));
1302 
1303       // check data belong to the correct region,every scan should get one row.
1304       for (int i = 0; i < ROWKEYS.length; i++) {
1305         if (i != ROWKEYS.length - 1) {
1306           assertEquals(1, countRows(ROWKEYS[i], ROWKEYS[i + 1]));
1307         } else {
1308           assertEquals(1, countRows(ROWKEYS[i], null));
1309         }
1310       }
1311 
1312     } finally {
1313       cleanupTable(table);
1314     }
1315   }
1316 
1317   /**
1318    * This creates and fixes a bad table with a region that is missing meta and
1319    * not assigned to a region server.
1320    */
1321   @Test (timeout=180000)
1322   public void testNotInMetaOrDeployedHole() throws Exception {
1323     TableName table =
1324         TableName.valueOf("tableNotInMetaOrDeployedHole");
1325     try {
1326       setupTable(table);
1327       assertEquals(ROWKEYS.length, countRows());
1328 
1329       // Mess it up by leaving a hole in the meta data
1330       admin.disableTable(table);
1331       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1332           Bytes.toBytes("C"), true, true, false); // don't rm from fs
1333       admin.enableTable(table);
1334 
1335       HBaseFsck hbck = doFsck(conf, false);
1336       assertErrors(hbck, new ERROR_CODE[] {
1337           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1338       // holes are separate from overlap groups
1339       assertEquals(0, hbck.getOverlapGroups(table).size());
1340 
1341       // fix hole
1342       assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
1343           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1344 
1345       // check that hole fixed
1346       assertNoErrors(doFsck(conf,false));
1347       assertEquals(ROWKEYS.length, countRows());
1348     } finally {
1349       cleanupTable(table);
1350     }
1351   }
1352 
1353   /**
1354    * This creates fixes a bad table with a hole in meta.
1355    */
1356   @Test (timeout=180000)
1357   public void testNotInMetaHole() throws Exception {
1358     TableName table =
1359         TableName.valueOf("tableNotInMetaHole");
1360     try {
1361       setupTable(table);
1362       assertEquals(ROWKEYS.length, countRows());
1363 
1364       // Mess it up by leaving a hole in the meta data
1365       admin.disableTable(table);
1366       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1367           Bytes.toBytes("C"), false, true, false); // don't rm from fs
1368       admin.enableTable(table);
1369 
1370       HBaseFsck hbck = doFsck(conf, false);
1371       assertErrors(hbck, new ERROR_CODE[] {
1372           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1373       // holes are separate from overlap groups
1374       assertEquals(0, hbck.getOverlapGroups(table).size());
1375 
1376       // fix hole
1377       assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
1378           ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1379 
1380       // check that hole fixed
1381       assertNoErrors(doFsck(conf,false));
1382       assertEquals(ROWKEYS.length, countRows());
1383     } finally {
1384       cleanupTable(table);
1385     }
1386   }
1387 
1388   /**
1389    * This creates and fixes a bad table with a region that is in meta but has
1390    * no deployment or data hdfs
1391    */
1392   @Test (timeout=180000)
1393   public void testNotInHdfs() throws Exception {
1394     TableName table =
1395         TableName.valueOf("tableNotInHdfs");
1396     try {
1397       setupTable(table);
1398       assertEquals(ROWKEYS.length, countRows());
1399 
1400       // make sure data in regions, if in wal only there is no data loss
1401       admin.flush(table);
1402 
1403       // Mess it up by leaving a hole in the hdfs data
1404       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1405           Bytes.toBytes("C"), false, false, true); // don't rm meta
1406 
1407       HBaseFsck hbck = doFsck(conf, false);
1408       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
1409       // holes are separate from overlap groups
1410       assertEquals(0, hbck.getOverlapGroups(table).size());
1411 
1412       // fix hole
1413       doFsck(conf, true);
1414 
1415       // check that hole fixed
1416       assertNoErrors(doFsck(conf,false));
1417       assertEquals(ROWKEYS.length - 2, countRows());
1418     } finally {
1419       cleanupTable(table);
1420     }
1421   }
1422 
1423   /**
1424    * This creates and fixes a bad table with a region that is in meta but has
1425    * no deployment or data hdfs. The table has region_replication set to 2.
1426    */
1427   @Test (timeout=180000)
1428   public void testNotInHdfsWithReplicas() throws Exception {
1429     TableName table =
1430         TableName.valueOf("tableNotInHdfs");
1431     HBaseAdmin admin = new HBaseAdmin(conf);
1432     try {
1433       HRegionInfo[] oldHris = new HRegionInfo[2];
1434       setupTableWithRegionReplica(table, 2);
1435       assertEquals(ROWKEYS.length, countRows());
1436       NavigableMap<HRegionInfo, ServerName> map = MetaScanner.allTableRegions(TEST_UTIL.getConnection(),
1437           tbl.getName());
1438       int i = 0;
1439       // store the HRIs of the regions we will mess up
1440       for (Map.Entry<HRegionInfo, ServerName> m : map.entrySet()) {
1441         if (m.getKey().getStartKey().length > 0 &&
1442             m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) {
1443           LOG.debug("Initially server hosting " + m.getKey() + " is " + m.getValue());
1444           oldHris[i++] = m.getKey();
1445         }
1446       }
1447       // make sure data in regions
1448       TEST_UTIL.getHBaseAdmin().flush(table.getName());
1449 
1450       // Mess it up by leaving a hole in the hdfs data
1451       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1452           Bytes.toBytes("C"), false, false, true); // don't rm meta
1453 
1454       HBaseFsck hbck = doFsck(conf, false);
1455       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
1456 
1457       // fix hole
1458       doFsck(conf, true);
1459 
1460       // check that hole fixed
1461       assertNoErrors(doFsck(conf,false));
1462       assertEquals(ROWKEYS.length - 2, countRows());
1463 
1464       // the following code checks whether the old primary/secondary has
1465       // been unassigned and the new primary/secondary has been assigned
1466       i = 0;
1467       HRegionInfo[] newHris = new HRegionInfo[2];
1468       // get all table's regions from meta
1469       map = MetaScanner.allTableRegions(TEST_UTIL.getConnection(), tbl.getName());
1470       // get the HRIs of the new regions (hbck created new regions for fixing the hdfs mess-up)
1471       for (Map.Entry<HRegionInfo, ServerName> m : map.entrySet()) {
1472         if (m.getKey().getStartKey().length > 0 &&
1473             m.getKey().getStartKey()[0] == Bytes.toBytes("B")[0]) {
1474           newHris[i++] = m.getKey();
1475         }
1476       }
1477       // get all the online regions in the regionservers
1478       Collection<ServerName> servers = admin.getClusterStatus().getServers();
1479       Set<HRegionInfo> onlineRegions = new HashSet<HRegionInfo>();
1480       for (ServerName s : servers) {
1481         List<HRegionInfo> list = admin.getOnlineRegions(s);
1482         onlineRegions.addAll(list);
1483       }
1484       // the new HRIs must be a subset of the online regions
1485       assertTrue(onlineRegions.containsAll(Arrays.asList(newHris)));
1486       // the old HRIs must not be part of the set (removeAll would return false if
1487       // the set didn't change)
1488       assertFalse(onlineRegions.removeAll(Arrays.asList(oldHris)));
1489     } finally {
1490       cleanupTable(table);
1491       admin.close();
1492     }
1493   }
1494 
1495 
1496   /**
1497    * This creates entries in hbase:meta with no hdfs data.  This should cleanly
1498    * remove the table.
1499    */
1500   @Test (timeout=180000)
1501   public void testNoHdfsTable() throws Exception {
1502     TableName table = TableName.valueOf("NoHdfsTable");
1503     setupTable(table);
1504     assertEquals(ROWKEYS.length, countRows());
1505 
1506     // make sure data in regions, if in wal only there is no data loss
1507     admin.flush(table);
1508 
1509     // Mess it up by deleting hdfs dirs
1510     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes(""),
1511         Bytes.toBytes("A"), false, false, true); // don't rm meta
1512     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1513         Bytes.toBytes("B"), false, false, true); // don't rm meta
1514     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1515         Bytes.toBytes("C"), false, false, true); // don't rm meta
1516     deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("C"),
1517         Bytes.toBytes(""), false, false, true); // don't rm meta
1518 
1519     // also remove the table directory in hdfs
1520     deleteTableDir(table);
1521 
1522     HBaseFsck hbck = doFsck(conf, false);
1523     assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS,
1524         ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS,
1525         ERROR_CODE.NOT_IN_HDFS,});
1526     // holes are separate from overlap groups
1527     assertEquals(0, hbck.getOverlapGroups(table).size());
1528 
1529     // fix hole
1530     doFsck(conf, true); // detect dangling regions and remove those
1531 
1532     // check that hole fixed
1533     assertNoErrors(doFsck(conf,false));
1534     assertFalse("Table " + table + " should have been deleted", admin.tableExists(table));
1535   }
1536 
1537   public void deleteTableDir(TableName table) throws IOException {
1538     Path rootDir = FSUtils.getRootDir(conf);
1539     FileSystem fs = rootDir.getFileSystem(conf);
1540     Path p = FSUtils.getTableDir(rootDir, table);
1541     HBaseFsck.debugLsr(conf, p);
1542     boolean success = fs.delete(p, true);
1543     LOG.info("Deleted " + p + " sucessfully? " + success);
1544   }
1545 
1546   /**
1547    * when the hbase.version file missing, It is fix the fault.
1548    */
1549   @Test (timeout=180000)
1550   public void testNoVersionFile() throws Exception {
1551     // delete the hbase.version file
1552     Path rootDir = FSUtils.getRootDir(conf);
1553     FileSystem fs = rootDir.getFileSystem(conf);
1554     Path versionFile = new Path(rootDir, HConstants.VERSION_FILE_NAME);
1555     fs.delete(versionFile, true);
1556 
1557     // test
1558     HBaseFsck hbck = doFsck(conf, false);
1559     assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_VERSION_FILE });
1560     // fix hbase.version missing
1561     doFsck(conf, true);
1562 
1563     // no version file fixed
1564     assertNoErrors(doFsck(conf, false));
1565   }
1566 
1567   /**
1568    * The region is not deployed when the table is disabled.
1569    */
1570   @Test (timeout=180000)
1571   public void testRegionShouldNotBeDeployed() throws Exception {
1572     TableName table =
1573         TableName.valueOf("tableRegionShouldNotBeDeployed");
1574     try {
1575       LOG.info("Starting testRegionShouldNotBeDeployed.");
1576       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
1577       assertTrue(cluster.waitForActiveAndReadyMaster());
1578 
1579 
1580       byte[][] SPLIT_KEYS = new byte[][] { new byte[0], Bytes.toBytes("aaa"),
1581           Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), Bytes.toBytes("ddd") };
1582       HTableDescriptor htdDisabled = new HTableDescriptor(table);
1583       htdDisabled.addFamily(new HColumnDescriptor(FAM));
1584 
1585       // Write the .tableinfo
1586       FSTableDescriptors fstd = new FSTableDescriptors(conf);
1587       fstd.createTableDescriptor(htdDisabled);
1588       List<HRegionInfo> disabledRegions =
1589           TEST_UTIL.createMultiRegionsInMeta(conf, htdDisabled, SPLIT_KEYS);
1590 
1591       // Let's just assign everything to first RS
1592       HRegionServer hrs = cluster.getRegionServer(0);
1593 
1594       // Create region files.
1595       admin.disableTable(table);
1596       admin.enableTable(table);
1597 
1598       // Disable the table and close its regions
1599       admin.disableTable(table);
1600       HRegionInfo region = disabledRegions.remove(0);
1601       byte[] regionName = region.getRegionName();
1602 
1603       // The region should not be assigned currently
1604       assertTrue(cluster.getServerWith(regionName) == -1);
1605 
1606       // Directly open a region on a region server.
1607       // If going through AM/ZK, the region won't be open.
1608       // Even it is opened, AM will close it which causes
1609       // flakiness of this test.
1610       HRegion r = HRegion.openHRegion(
1611         region, htdDisabled, hrs.getWAL(region), conf);
1612       hrs.addToOnlineRegions(r);
1613 
1614       HBaseFsck hbck = doFsck(conf, false);
1615       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.SHOULD_NOT_BE_DEPLOYED });
1616 
1617       // fix this fault
1618       doFsck(conf, true);
1619 
1620       // check result
1621       assertNoErrors(doFsck(conf, false));
1622     } finally {
1623       admin.enableTable(table);
1624       cleanupTable(table);
1625     }
1626   }
1627 
1628   /**
1629    * This creates two tables and mess both of them and fix them one by one
1630    */
1631   @Test (timeout=180000)
1632   public void testFixByTable() throws Exception {
1633     TableName table1 =
1634         TableName.valueOf("testFixByTable1");
1635     TableName table2 =
1636         TableName.valueOf("testFixByTable2");
1637     try {
1638       setupTable(table1);
1639       // make sure data in regions, if in wal only there is no data loss
1640       admin.flush(table1);
1641       // Mess them up by leaving a hole in the hdfs data
1642       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1643         Bytes.toBytes("C"), false, false, true); // don't rm meta
1644 
1645       setupTable(table2);
1646       // make sure data in regions, if in wal only there is no data loss
1647       admin.flush(table2);
1648       // Mess them up by leaving a hole in the hdfs data
1649       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1650         Bytes.toBytes("C"), false, false, true); // don't rm meta
1651 
1652       HBaseFsck hbck = doFsck(conf, false);
1653       assertErrors(hbck, new ERROR_CODE[] {
1654         ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS});
1655 
1656       // fix hole in table 1
1657       doFsck(conf, true, table1);
1658       // check that hole in table 1 fixed
1659       assertNoErrors(doFsck(conf, false, table1));
1660       // check that hole in table 2 still there
1661       assertErrors(doFsck(conf, false, table2),
1662         new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
1663 
1664       // fix hole in table 2
1665       doFsck(conf, true, table2);
1666       // check that hole in both tables fixed
1667       assertNoErrors(doFsck(conf, false));
1668       assertEquals(ROWKEYS.length - 2, countRows());
1669     } finally {
1670       cleanupTable(table1);
1671       cleanupTable(table2);
1672     }
1673   }
1674   /**
1675    * A split parent in meta, in hdfs, and not deployed
1676    */
1677   @Test (timeout=180000)
1678   public void testLingeringSplitParent() throws Exception {
1679     TableName table =
1680         TableName.valueOf("testLingeringSplitParent");
1681     Table meta = null;
1682     try {
1683       setupTable(table);
1684       assertEquals(ROWKEYS.length, countRows());
1685 
1686       // make sure data in regions, if in wal only there is no data loss
1687       admin.flush(table);
1688       HRegionLocation location = tbl.getRegionLocation("B");
1689 
1690       // Delete one region from meta, but not hdfs, unassign it.
1691       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
1692         Bytes.toBytes("C"), true, true, false);
1693 
1694       // Create a new meta entry to fake it as a split parent.
1695       meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
1696       HRegionInfo hri = location.getRegionInfo();
1697 
1698       HRegionInfo a = new HRegionInfo(tbl.getName(),
1699         Bytes.toBytes("B"), Bytes.toBytes("BM"));
1700       HRegionInfo b = new HRegionInfo(tbl.getName(),
1701         Bytes.toBytes("BM"), Bytes.toBytes("C"));
1702 
1703       hri.setOffline(true);
1704       hri.setSplit(true);
1705 
1706       MetaTableAccessor.addRegionToMeta(meta, hri, a, b);
1707       meta.close();
1708       admin.flush(TableName.META_TABLE_NAME);
1709 
1710       HBaseFsck hbck = doFsck(conf, false);
1711       assertErrors(hbck, new ERROR_CODE[] {
1712         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1713 
1714       // regular repair cannot fix lingering split parent
1715       hbck = doFsck(conf, true);
1716       assertErrors(hbck, new ERROR_CODE[] {
1717         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN });
1718       assertFalse(hbck.shouldRerun());
1719       hbck = doFsck(conf, false);
1720       assertErrors(hbck, new ERROR_CODE[] {
1721         ERROR_CODE.LINGERING_SPLIT_PARENT, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1722 
1723       // fix lingering split parent
1724       hbck = new HBaseFsck(conf, hbfsckExecutorService);
1725       hbck.connect();
1726       hbck.setDisplayFullReport(); // i.e. -details
1727       hbck.setTimeLag(0);
1728       hbck.setFixSplitParents(true);
1729       hbck.onlineHbck();
1730       assertTrue(hbck.shouldRerun());
1731       hbck.close();
1732 
1733       Get get = new Get(hri.getRegionName());
1734       Result result = meta.get(get);
1735       assertTrue(result.getColumnCells(HConstants.CATALOG_FAMILY,
1736         HConstants.SPLITA_QUALIFIER).isEmpty());
1737       assertTrue(result.getColumnCells(HConstants.CATALOG_FAMILY,
1738         HConstants.SPLITB_QUALIFIER).isEmpty());
1739       admin.flush(TableName.META_TABLE_NAME);
1740 
1741       // fix other issues
1742       doFsck(conf, true);
1743 
1744       // check that all are fixed
1745       assertNoErrors(doFsck(conf, false));
1746       assertEquals(ROWKEYS.length, countRows());
1747     } finally {
1748       cleanupTable(table);
1749       IOUtils.closeQuietly(meta);
1750     }
1751   }
1752 
1753   /**
1754    * Tests that LINGERING_SPLIT_PARENT is not erroneously reported for
1755    * valid cases where the daughters are there.
1756    */
1757   @Test (timeout=180000)
1758   public void testValidLingeringSplitParent() throws Exception {
1759     TableName table =
1760         TableName.valueOf("testLingeringSplitParent");
1761     Table meta = null;
1762     try {
1763       setupTable(table);
1764       assertEquals(ROWKEYS.length, countRows());
1765 
1766       // make sure data in regions, if in wal only there is no data loss
1767       admin.flush(table);
1768       HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
1769 
1770       meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
1771       HRegionInfo hri = location.getRegionInfo();
1772 
1773       // do a regular split
1774       byte[] regionName = location.getRegionInfo().getRegionName();
1775       admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
1776       TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
1777 
1778       // TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
1779       // for some time until children references are deleted. HBCK erroneously sees this as
1780       // overlapping regions
1781       HBaseFsck hbck = doFsck(
1782         conf, true, true, false, false, false, true, true, true, false, false, false, null);
1783       assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
1784 
1785       // assert that the split hbase:meta entry is still there.
1786       Get get = new Get(hri.getRegionName());
1787       Result result = meta.get(get);
1788       assertNotNull(result);
1789       assertNotNull(MetaTableAccessor.getHRegionInfo(result));
1790 
1791       assertEquals(ROWKEYS.length, countRows());
1792 
1793       // assert that we still have the split regions
1794       assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
1795       assertNoErrors(doFsck(conf, false));
1796     } finally {
1797       cleanupTable(table);
1798       IOUtils.closeQuietly(meta);
1799     }
1800   }
1801 
1802   /**
1803    * Split crashed after write to hbase:meta finished for the parent region, but
1804    * failed to write daughters (pre HBASE-7721 codebase)
1805    */
1806   @Test(timeout=75000)
1807   public void testSplitDaughtersNotInMeta() throws Exception {
1808     TableName table = TableName.valueOf("testSplitdaughtersNotInMeta");
1809     Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
1810     try {
1811       setupTable(table);
1812       assertEquals(ROWKEYS.length, countRows());
1813 
1814       // make sure data in regions, if in wal only there is no data loss
1815       admin.flush(table);
1816       HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
1817 
1818       HRegionInfo hri = location.getRegionInfo();
1819 
1820       // do a regular split
1821       byte[] regionName = location.getRegionInfo().getRegionName();
1822       admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
1823       TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
1824 
1825       PairOfSameType<HRegionInfo> daughters =
1826           MetaTableAccessor.getDaughterRegions(meta.get(new Get(regionName)));
1827 
1828       // Delete daughter regions from meta, but not hdfs, unassign it.
1829       Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
1830       undeployRegion(connection, hris.get(daughters.getFirst()), daughters.getFirst());
1831       undeployRegion(connection, hris.get(daughters.getSecond()), daughters.getSecond());
1832 
1833       List<Delete> deletes = new ArrayList<>();
1834       deletes.add(new Delete(daughters.getFirst().getRegionName()));
1835       deletes.add(new Delete(daughters.getSecond().getRegionName()));
1836       meta.delete(deletes);
1837 
1838       // Remove daughters from regionStates
1839       RegionStates regionStates = TEST_UTIL.getMiniHBaseCluster().getMaster().
1840           getAssignmentManager().getRegionStates();
1841       regionStates.deleteRegion(daughters.getFirst());
1842       regionStates.deleteRegion(daughters.getSecond());
1843 
1844       HBaseFsck hbck = doFsck(conf, false);
1845       assertErrors(hbck,
1846           new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1847               ERROR_CODE.HOLE_IN_REGION_CHAIN }); //no LINGERING_SPLIT_PARENT
1848 
1849       // now fix it. The fix should not revert the region split, but add daughters to META
1850       hbck = doFsck(
1851         conf, true, true, false, false, false, false, false, false, false, false, false, null);
1852       assertErrors(hbck,
1853           new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
1854               ERROR_CODE.HOLE_IN_REGION_CHAIN });
1855 
1856       // assert that the split hbase:meta entry is still there.
1857       Get get = new Get(hri.getRegionName());
1858       Result result = meta.get(get);
1859       assertNotNull(result);
1860       assertNotNull(MetaTableAccessor.getHRegionInfo(result));
1861 
1862       assertEquals(ROWKEYS.length, countRows());
1863 
1864       // assert that we still have the split regions
1865       assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
1866       assertNoErrors(doFsck(conf, false)); //should be fixed by now
1867     } finally {
1868       meta.close();
1869       cleanupTable(table);
1870     }
1871   }
1872 
1873   /**
1874    * This creates and fixes a bad table with a missing region which is the 1st region -- hole in
1875    * meta and data missing in the fs.
1876    */
1877   @Test(timeout=120000)
1878   public void testMissingFirstRegion() throws Exception {
1879     TableName table = TableName.valueOf("testMissingFirstRegion");
1880     try {
1881       setupTable(table);
1882       assertEquals(ROWKEYS.length, countRows());
1883 
1884       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1885       admin.disableTable(table);
1886       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes(""), Bytes.toBytes("A"), true,
1887           true, true);
1888       admin.enableTable(table);
1889 
1890       HBaseFsck hbck = doFsck(conf, false);
1891       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.FIRST_REGION_STARTKEY_NOT_EMPTY });
1892       // fix hole
1893       doFsck(conf, true);
1894       // check that hole fixed
1895       assertNoErrors(doFsck(conf, false));
1896     } finally {
1897       cleanupTable(table);
1898     }
1899   }
1900 
1901   /**
1902    * This creates and fixes a bad table with a missing region which is the 1st region -- hole in
1903    * meta and data missing in the fs.
1904    */
1905   @Test(timeout=120000)
1906   public void testRegionDeployedNotInHdfs() throws Exception {
1907     TableName table =
1908         TableName.valueOf("testSingleRegionDeployedNotInHdfs");
1909     try {
1910       setupTable(table);
1911       admin.flush(table);
1912 
1913       // Mess it up by deleting region dir
1914       deleteRegion(conf, tbl.getTableDescriptor(),
1915         HConstants.EMPTY_START_ROW, Bytes.toBytes("A"), false,
1916         false, true);
1917 
1918       HBaseFsck hbck = doFsck(conf, false);
1919       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
1920       // fix hole
1921       doFsck(conf, true);
1922       // check that hole fixed
1923       assertNoErrors(doFsck(conf, false));
1924     } finally {
1925       cleanupTable(table);
1926     }
1927   }
1928 
1929   /**
1930    * This creates and fixes a bad table with missing last region -- hole in meta and data missing in
1931    * the fs.
1932    */
1933   @Test(timeout=120000)
1934   public void testMissingLastRegion() throws Exception {
1935     TableName table =
1936         TableName.valueOf("testMissingLastRegion");
1937     try {
1938       setupTable(table);
1939       assertEquals(ROWKEYS.length, countRows());
1940 
1941       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
1942       admin.disableTable(table);
1943       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("C"), Bytes.toBytes(""), true,
1944           true, true);
1945       admin.enableTable(table);
1946 
1947       HBaseFsck hbck = doFsck(conf, false);
1948       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY });
1949       // fix hole
1950       doFsck(conf, true);
1951       // check that hole fixed
1952       assertNoErrors(doFsck(conf, false));
1953     } finally {
1954       cleanupTable(table);
1955     }
1956   }
1957 
1958   /**
1959    * Test -noHdfsChecking option can detect and fix assignments issue.
1960    */
1961   @Test (timeout=180000)
1962   public void testFixAssignmentsAndNoHdfsChecking() throws Exception {
1963     TableName table =
1964         TableName.valueOf("testFixAssignmentsAndNoHdfsChecking");
1965     try {
1966       setupTable(table);
1967       assertEquals(ROWKEYS.length, countRows());
1968 
1969       // Mess it up by closing a region
1970       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
1971         Bytes.toBytes("B"), true, false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
1972 
1973       // verify there is no other errors
1974       HBaseFsck hbck = doFsck(conf, false);
1975       assertErrors(hbck, new ERROR_CODE[] {
1976         ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1977 
1978       // verify that noHdfsChecking report the same errors
1979       HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
1980       fsck.connect();
1981       fsck.setDisplayFullReport(); // i.e. -details
1982       fsck.setTimeLag(0);
1983       fsck.setCheckHdfs(false);
1984       fsck.onlineHbck();
1985       assertErrors(fsck, new ERROR_CODE[] {
1986         ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
1987       fsck.close();
1988 
1989       // verify that fixAssignments works fine with noHdfsChecking
1990       fsck = new HBaseFsck(conf, hbfsckExecutorService);
1991       fsck.connect();
1992       fsck.setDisplayFullReport(); // i.e. -details
1993       fsck.setTimeLag(0);
1994       fsck.setCheckHdfs(false);
1995       fsck.setFixAssignments(true);
1996       fsck.onlineHbck();
1997       assertTrue(fsck.shouldRerun());
1998       fsck.onlineHbck();
1999       assertNoErrors(fsck);
2000 
2001       assertEquals(ROWKEYS.length, countRows());
2002 
2003       fsck.close();
2004     } finally {
2005       cleanupTable(table);
2006     }
2007   }
2008 
2009   /**
2010    * Test -noHdfsChecking option can detect region is not in meta but deployed.
2011    * However, it can not fix it without checking Hdfs because we need to get
2012    * the region info from Hdfs in this case, then to patch the meta.
2013    */
2014   @Test (timeout=180000)
2015   public void testFixMetaNotWorkingWithNoHdfsChecking() throws Exception {
2016     TableName table =
2017         TableName.valueOf("testFixMetaNotWorkingWithNoHdfsChecking");
2018     try {
2019       setupTable(table);
2020       assertEquals(ROWKEYS.length, countRows());
2021 
2022       // Mess it up by deleting a region from the metadata
2023       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
2024         Bytes.toBytes("B"), false, true, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
2025 
2026       // verify there is no other errors
2027       HBaseFsck hbck = doFsck(conf, false);
2028       assertErrors(hbck,
2029           new ERROR_CODE[] { ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN });
2030 
2031       // verify that noHdfsChecking report the same errors
2032       HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
2033       fsck.connect();
2034       fsck.setDisplayFullReport(); // i.e. -details
2035       fsck.setTimeLag(0);
2036       fsck.setCheckHdfs(false);
2037       fsck.onlineHbck();
2038       assertErrors(fsck,
2039           new ERROR_CODE[] { ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN });
2040       fsck.close();
2041 
2042       // verify that fixMeta doesn't work with noHdfsChecking
2043       fsck = new HBaseFsck(conf, hbfsckExecutorService);
2044       fsck.connect();
2045       fsck.setDisplayFullReport(); // i.e. -details
2046       fsck.setTimeLag(0);
2047       fsck.setCheckHdfs(false);
2048       fsck.setFixAssignments(true);
2049       fsck.setFixMeta(true);
2050       fsck.onlineHbck();
2051       assertFalse(fsck.shouldRerun());
2052       assertErrors(fsck,
2053           new ERROR_CODE[] { ERROR_CODE.NOT_IN_META, ERROR_CODE.HOLE_IN_REGION_CHAIN });
2054       fsck.close();
2055 
2056       // fix the cluster so other tests won't be impacted
2057       fsck = doFsck(conf, true);
2058       assertTrue(fsck.shouldRerun());
2059       fsck = doFsck(conf, true);
2060       assertNoErrors(fsck);
2061     } finally {
2062       cleanupTable(table);
2063     }
2064   }
2065 
2066   /**
2067    * Test -fixHdfsHoles doesn't work with -noHdfsChecking option,
2068    * and -noHdfsChecking can't detect orphan Hdfs region.
2069    */
2070   @Test (timeout=180000)
2071   public void testFixHdfsHolesNotWorkingWithNoHdfsChecking() throws Exception {
2072     TableName table =
2073         TableName.valueOf("testFixHdfsHolesNotWorkingWithNoHdfsChecking");
2074     try {
2075       setupTable(table);
2076       assertEquals(ROWKEYS.length, countRows());
2077 
2078       // Mess it up by creating an overlap in the metadata
2079       admin.disableTable(table);
2080       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
2081         Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
2082       TEST_UTIL.getHBaseAdmin().enableTable(table);
2083 
2084       HRegionInfo hriOverlap =
2085           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A2"), Bytes.toBytes("B"));
2086       TEST_UTIL.getHBaseCluster().getMaster().assignRegion(hriOverlap);
2087       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager()
2088         .waitForAssignment(hriOverlap);
2089       ServerName server = regionStates.getRegionServerOfRegion(hriOverlap);
2090       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
2091 
2092       HBaseFsck hbck = doFsck(conf, false);
2093       assertErrors(hbck, new ERROR_CODE[] {
2094         ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
2095         ERROR_CODE.HOLE_IN_REGION_CHAIN});
2096 
2097       // verify that noHdfsChecking can't detect ORPHAN_HDFS_REGION
2098       HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
2099       fsck.connect();
2100       fsck.setDisplayFullReport(); // i.e. -details
2101       fsck.setTimeLag(0);
2102       fsck.setCheckHdfs(false);
2103       fsck.onlineHbck();
2104       assertErrors(fsck, new ERROR_CODE[] {
2105         ERROR_CODE.HOLE_IN_REGION_CHAIN});
2106       fsck.close();
2107 
2108       // verify that fixHdfsHoles doesn't work with noHdfsChecking
2109       fsck = new HBaseFsck(conf, hbfsckExecutorService);
2110       fsck.connect();
2111       fsck.setDisplayFullReport(); // i.e. -details
2112       fsck.setTimeLag(0);
2113       fsck.setCheckHdfs(false);
2114       fsck.setFixHdfsHoles(true);
2115       fsck.setFixHdfsOverlaps(true);
2116       fsck.setFixHdfsOrphans(true);
2117       fsck.onlineHbck();
2118       assertFalse(fsck.shouldRerun());
2119       assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN});
2120       fsck.close();
2121     } finally {
2122       if (admin.isTableDisabled(table)) {
2123         admin.enableTable(table);
2124       }
2125       cleanupTable(table);
2126     }
2127   }
2128 
2129   /**
2130    * We don't have an easy way to verify that a flush completed, so we loop until we find a
2131    * legitimate hfile and return it.
2132    * @param fs
2133    * @param table
2134    * @return Path of a flushed hfile.
2135    * @throws IOException
2136    */
2137   Path getFlushedHFile(FileSystem fs, TableName table) throws IOException {
2138     Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
2139     Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
2140     Path famDir = new Path(regionDir, FAM_STR);
2141 
2142     // keep doing this until we get a legit hfile
2143     while (true) {
2144       FileStatus[] hfFss = fs.listStatus(famDir);
2145       if (hfFss.length == 0) {
2146         continue;
2147       }
2148       for (FileStatus hfs : hfFss) {
2149         if (!hfs.isDirectory()) {
2150           return hfs.getPath();
2151         }
2152       }
2153     }
2154   }
2155 
2156   /**
2157    * Gets flushed mob files.
2158    * @param fs The current file system.
2159    * @param table The current table name.
2160    * @return Path of a flushed hfile.
2161    * @throws IOException
2162    */
2163   Path getFlushedMobFile(FileSystem fs, TableName table) throws IOException {
2164     Path regionDir = MobUtils.getMobRegionPath(conf, table);
2165     Path famDir = new Path(regionDir, FAM_STR);
2166 
2167     // keep doing this until we get a legit hfile
2168     while (true) {
2169       FileStatus[] hfFss = fs.listStatus(famDir);
2170       if (hfFss.length == 0) {
2171         continue;
2172       }
2173       for (FileStatus hfs : hfFss) {
2174         if (!hfs.isDirectory()) {
2175           return hfs.getPath();
2176         }
2177       }
2178     }
2179   }
2180 
2181   /**
2182    * Creates a new mob file name by the old one.
2183    * @param oldFileName The old mob file name.
2184    * @return The new mob file name.
2185    */
2186   String createMobFileName(String oldFileName) {
2187     MobFileName mobFileName = MobFileName.create(oldFileName);
2188     String startKey = mobFileName.getStartKey();
2189     String date = mobFileName.getDate();
2190     return MobFileName.create(startKey, date, UUID.randomUUID().toString().replaceAll("-", ""))
2191       .getFileName();
2192   }
2193 
2194   /**
2195    * This creates a table and then corrupts an hfile.  Hbck should quarantine the file.
2196    */
2197   @Test(timeout=180000)
2198   public void testQuarantineCorruptHFile() throws Exception {
2199     TableName table = TableName.valueOf(name.getMethodName());
2200     try {
2201       setupTable(table);
2202       assertEquals(ROWKEYS.length, countRows());
2203       admin.flush(table); // flush is async.
2204 
2205       FileSystem fs = FileSystem.get(conf);
2206       Path hfile = getFlushedHFile(fs, table);
2207 
2208       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
2209       admin.disableTable(table);
2210 
2211       // create new corrupt file called deadbeef (valid hfile name)
2212       Path corrupt = new Path(hfile.getParent(), "deadbeef");
2213       TestHFile.truncateFile(fs, hfile, corrupt);
2214       LOG.info("Created corrupted file " + corrupt);
2215       HBaseFsck.debugLsr(conf, FSUtils.getRootDir(conf));
2216 
2217       // we cannot enable here because enable never finished due to the corrupt region.
2218       HBaseFsck res = HbckTestingUtil.doHFileQuarantine(conf, table);
2219       assertEquals(res.getRetCode(), 0);
2220       HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
2221       assertEquals(hfcc.getHFilesChecked(), 5);
2222       assertEquals(hfcc.getCorrupted().size(), 1);
2223       assertEquals(hfcc.getFailures().size(), 0);
2224       assertEquals(hfcc.getQuarantined().size(), 1);
2225       assertEquals(hfcc.getMissing().size(), 0);
2226 
2227       // Its been fixed, verify that we can enable.
2228       admin.enableTable(table);
2229     } finally {
2230       cleanupTable(table);
2231     }
2232   }
2233 
2234   /**
2235    * This creates a table and then corrupts a mob file.  Hbck should quarantine the file.
2236    */
2237   @Test(timeout=180000)
2238   public void testQuarantineCorruptMobFile() throws Exception {
2239     TableName table = TableName.valueOf(name.getMethodName());
2240     try {
2241       setupMobTable(table);
2242       assertEquals(ROWKEYS.length, countRows());
2243       admin.flush(table);
2244 
2245       FileSystem fs = FileSystem.get(conf);
2246       Path mobFile = getFlushedMobFile(fs, table);
2247       admin.disableTable(table);
2248       // create new corrupt mob file.
2249       String corruptMobFile = createMobFileName(mobFile.getName());
2250       Path corrupt = new Path(mobFile.getParent(), corruptMobFile);
2251       TestHFile.truncateFile(fs, mobFile, corrupt);
2252       LOG.info("Created corrupted mob file " + corrupt);
2253       HBaseFsck.debugLsr(conf, FSUtils.getRootDir(conf));
2254       HBaseFsck.debugLsr(conf, MobUtils.getMobHome(conf));
2255 
2256       // A corrupt mob file doesn't abort the start of regions, so we can enable the table.
2257       admin.enableTable(table);
2258       HBaseFsck res = HbckTestingUtil.doHFileQuarantine(conf, table);
2259       assertEquals(res.getRetCode(), 0);
2260       HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
2261       assertEquals(hfcc.getHFilesChecked(), 4);
2262       assertEquals(hfcc.getCorrupted().size(), 0);
2263       assertEquals(hfcc.getFailures().size(), 0);
2264       assertEquals(hfcc.getQuarantined().size(), 0);
2265       assertEquals(hfcc.getMissing().size(), 0);
2266       assertEquals(hfcc.getMobFilesChecked(), 5);
2267       assertEquals(hfcc.getCorruptedMobFiles().size(), 1);
2268       assertEquals(hfcc.getFailureMobFiles().size(), 0);
2269       assertEquals(hfcc.getQuarantinedMobFiles().size(), 1);
2270       assertEquals(hfcc.getMissedMobFiles().size(), 0);
2271       String quarantinedMobFile = hfcc.getQuarantinedMobFiles().iterator().next().getName();
2272       assertEquals(corruptMobFile, quarantinedMobFile);
2273     } finally {
2274       cleanupTable(table);
2275     }
2276   }
2277 
2278   /**
2279    * Test that use this should have a timeout, because this method could potentially wait forever.
2280   */
2281   private void doQuarantineTest(TableName table, HBaseFsck hbck, int check,
2282                                 int corrupt, int fail, int quar, int missing) throws Exception {
2283     try {
2284       setupTable(table);
2285       assertEquals(ROWKEYS.length, countRows());
2286       admin.flush(table); // flush is async.
2287 
2288       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
2289       admin.disableTable(table);
2290 
2291       String[] args = {"-sidelineCorruptHFiles", "-repairHoles", "-ignorePreCheckPermission",
2292           table.getNameAsString()};
2293       HBaseFsck res = hbck.exec(hbfsckExecutorService, args);
2294 
2295       HFileCorruptionChecker hfcc = res.getHFilecorruptionChecker();
2296       assertEquals(hfcc.getHFilesChecked(), check);
2297       assertEquals(hfcc.getCorrupted().size(), corrupt);
2298       assertEquals(hfcc.getFailures().size(), fail);
2299       assertEquals(hfcc.getQuarantined().size(), quar);
2300       assertEquals(hfcc.getMissing().size(), missing);
2301 
2302       // its been fixed, verify that we can enable
2303       admin.enableTableAsync(table);
2304       while (!admin.isTableEnabled(table)) {
2305         try {
2306           Thread.sleep(250);
2307         } catch (InterruptedException e) {
2308           e.printStackTrace();
2309           fail("Interrupted when trying to enable table " + table);
2310         }
2311       }
2312     } finally {
2313       cleanupTable(table);
2314     }
2315   }
2316 
2317   /**
2318    * This creates a table and simulates the race situation where a concurrent compaction or split
2319    * has removed an hfile after the corruption checker learned about it.
2320    */
2321   @Test(timeout=180000)
2322   public void testQuarantineMissingHFile() throws Exception {
2323     TableName table = TableName.valueOf(name.getMethodName());
2324 
2325     // inject a fault in the hfcc created.
2326     final FileSystem fs = FileSystem.get(conf);
2327     HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
2328       @Override
2329       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
2330         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
2331           AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
2332           @Override
2333           protected void checkHFile(Path p) throws IOException {
2334             if (attemptedFirstHFile.compareAndSet(false, true)) {
2335               assertTrue(fs.delete(p, true)); // make sure delete happened.
2336             }
2337             super.checkHFile(p);
2338           }
2339         };
2340       }
2341     };
2342     doQuarantineTest(table, hbck, 4, 0, 0, 0, 1); // 4 attempted, but 1 missing.
2343     hbck.close();
2344   }
2345 
2346   /**
2347    * This creates a table and simulates the race situation where a concurrent compaction or split
2348    * has removed an colfam dir before the corruption checker got to it.
2349    */
2350   // Disabled because fails sporadically.  Is this test right?  Timing-wise, there could be no
2351   // files in a column family on initial creation -- as suggested by Matteo.
2352   @Ignore @Test(timeout=180000)
2353   public void testQuarantineMissingFamdir() throws Exception {
2354     TableName table = TableName.valueOf(name.getMethodName());
2355     // inject a fault in the hfcc created.
2356     final FileSystem fs = FileSystem.get(conf);
2357     HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
2358       @Override
2359       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
2360         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
2361           AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
2362           @Override
2363           protected void checkColFamDir(Path p) throws IOException {
2364             if (attemptedFirstHFile.compareAndSet(false, true)) {
2365               assertTrue(fs.delete(p, true)); // make sure delete happened.
2366             }
2367             super.checkColFamDir(p);
2368           }
2369         };
2370       }
2371     };
2372     doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
2373     hbck.close();
2374   }
2375 
2376   /**
2377    * This creates a table and simulates the race situation where a concurrent compaction or split
2378    * has removed a region dir before the corruption checker got to it.
2379    */
2380   @Test(timeout=180000)
2381   public void testQuarantineMissingRegionDir() throws Exception {
2382     TableName table = TableName.valueOf(name.getMethodName());
2383     // inject a fault in the hfcc created.
2384     final FileSystem fs = FileSystem.get(conf);
2385     HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
2386       @Override
2387       public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
2388       throws IOException {
2389         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
2390           AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
2391           @Override
2392           protected void checkRegionDir(Path p) throws IOException {
2393             if (attemptedFirstHFile.compareAndSet(false, true)) {
2394               assertTrue(fs.delete(p, true)); // make sure delete happened.
2395             }
2396             super.checkRegionDir(p);
2397           }
2398         };
2399       }
2400     };
2401     doQuarantineTest(table, hbck, 3, 0, 0, 0, 1);
2402     hbck.close();
2403   }
2404 
2405   /**
2406    * Test fixing lingering reference file.
2407    */
2408   @Test (timeout=180000)
2409   public void testLingeringReferenceFile() throws Exception {
2410     TableName table =
2411         TableName.valueOf("testLingeringReferenceFile");
2412     try {
2413       setupTable(table);
2414       assertEquals(ROWKEYS.length, countRows());
2415 
2416       // Mess it up by creating a fake reference file
2417       FileSystem fs = FileSystem.get(conf);
2418       Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), table);
2419       Path regionDir = FSUtils.getRegionDirs(fs, tableDir).get(0);
2420       Path famDir = new Path(regionDir, FAM_STR);
2421       Path fakeReferenceFile = new Path(famDir, "fbce357483ceea.12144538");
2422       fs.create(fakeReferenceFile);
2423 
2424       HBaseFsck hbck = doFsck(conf, false);
2425       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.LINGERING_REFERENCE_HFILE });
2426       // fix reference file
2427       doFsck(conf, true);
2428       // check that reference file fixed
2429       assertNoErrors(doFsck(conf, false));
2430     } finally {
2431       cleanupTable(table);
2432     }
2433   }
2434 
2435   /**
2436    * Test mission REGIONINFO_QUALIFIER in hbase:meta
2437    */
2438   @Test (timeout=180000)
2439   public void testMissingRegionInfoQualifier() throws Exception {
2440     Connection connection = ConnectionFactory.createConnection(conf);
2441     TableName table = TableName.valueOf("testMissingRegionInfoQualifier");
2442     try {
2443       setupTable(table);
2444 
2445       // Mess it up by removing the RegionInfo for one region.
2446       final List<Delete> deletes = new LinkedList<Delete>();
2447       Table meta = connection.getTable(TableName.META_TABLE_NAME, hbfsckExecutorService);
2448       MetaScanner.metaScan(connection, new MetaScanner.MetaScannerVisitor() {
2449 
2450         @Override
2451         public boolean processRow(Result rowResult) throws IOException {
2452           HRegionInfo hri = MetaTableAccessor.getHRegionInfo(rowResult);
2453           if (hri != null && !hri.getTable().isSystemTable()) {
2454             Delete delete = new Delete(rowResult.getRow());
2455             delete.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
2456             deletes.add(delete);
2457           }
2458           return true;
2459         }
2460 
2461         @Override
2462         public void close() throws IOException {
2463         }
2464       });
2465       meta.delete(deletes);
2466 
2467       // Mess it up by creating a fake hbase:meta entry with no associated RegionInfo
2468       meta.put(new Put(Bytes.toBytes(table + ",,1361911384013.810e28f59a57da91c66")).add(
2469         HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER, Bytes.toBytes("node1:60020")));
2470       meta.put(new Put(Bytes.toBytes(table + ",,1361911384013.810e28f59a57da91c66")).add(
2471         HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER, Bytes.toBytes(1362150791183L)));
2472       meta.close();
2473 
2474       HBaseFsck hbck = doFsck(conf, false);
2475       assertTrue(hbck.getErrors().getErrorList().contains(ERROR_CODE.EMPTY_META_CELL));
2476 
2477       // fix reference file
2478       hbck = doFsck(conf, true);
2479 
2480       // check that reference file fixed
2481       assertFalse(hbck.getErrors().getErrorList().contains(ERROR_CODE.EMPTY_META_CELL));
2482     } finally {
2483       cleanupTable(table);
2484     }
2485     connection.close();
2486   }
2487 
2488   /**
2489    * Test pluggable error reporter. It can be plugged in
2490    * from system property or configuration.
2491    */
2492   @Test (timeout=180000)
2493   public void testErrorReporter() throws Exception {
2494     try {
2495       MockErrorReporter.calledCount = 0;
2496       doFsck(conf, false);
2497       assertEquals(MockErrorReporter.calledCount, 0);
2498 
2499       conf.set("hbasefsck.errorreporter", MockErrorReporter.class.getName());
2500       doFsck(conf, false);
2501       assertTrue(MockErrorReporter.calledCount > 20);
2502     } finally {
2503       conf.set("hbasefsck.errorreporter",
2504         PrintingErrorReporter.class.getName());
2505       MockErrorReporter.calledCount = 0;
2506     }
2507   }
2508 
2509   static class MockErrorReporter implements ErrorReporter {
2510     static int calledCount = 0;
2511 
2512     @Override
2513     public void clear() {
2514       calledCount++;
2515     }
2516 
2517     @Override
2518     public void report(String message) {
2519       calledCount++;
2520     }
2521 
2522     @Override
2523     public void reportError(String message) {
2524       calledCount++;
2525     }
2526 
2527     @Override
2528     public void reportError(ERROR_CODE errorCode, String message) {
2529       calledCount++;
2530     }
2531 
2532     @Override
2533     public void reportError(ERROR_CODE errorCode, String message, TableInfo table) {
2534       calledCount++;
2535     }
2536 
2537     @Override
2538     public void reportError(ERROR_CODE errorCode,
2539         String message, TableInfo table, HbckInfo info) {
2540       calledCount++;
2541     }
2542 
2543     @Override
2544     public void reportError(ERROR_CODE errorCode, String message,
2545         TableInfo table, HbckInfo info1, HbckInfo info2) {
2546       calledCount++;
2547     }
2548 
2549     @Override
2550     public int summarize() {
2551       return ++calledCount;
2552     }
2553 
2554     @Override
2555     public void detail(String details) {
2556       calledCount++;
2557     }
2558 
2559     @Override
2560     public ArrayList<ERROR_CODE> getErrorList() {
2561       calledCount++;
2562       return new ArrayList<ERROR_CODE>();
2563     }
2564 
2565     @Override
2566     public void progress() {
2567       calledCount++;
2568     }
2569 
2570     @Override
2571     public void print(String message) {
2572       calledCount++;
2573     }
2574 
2575     @Override
2576     public void resetErrors() {
2577       calledCount++;
2578     }
2579 
2580     @Override
2581     public boolean tableHasErrors(TableInfo table) {
2582       calledCount++;
2583       return false;
2584     }
2585   }
2586 
2587   @Test(timeout=180000)
2588   public void testCheckTableLocks() throws Exception {
2589     IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge(0);
2590     EnvironmentEdgeManager.injectEdge(edge);
2591     // check no errors
2592     HBaseFsck hbck = doFsck(conf, false);
2593     assertNoErrors(hbck);
2594 
2595     ServerName mockName = ServerName.valueOf("localhost", 60000, 1);
2596     final TableName tableName = TableName.valueOf("foo");
2597 
2598     // obtain one lock
2599     final TableLockManager tableLockManager =
2600       TableLockManager.createTableLockManager(conf, TEST_UTIL.getZooKeeperWatcher(), mockName);
2601     TableLock writeLock = tableLockManager.writeLock(tableName, "testCheckTableLocks");
2602     writeLock.acquire();
2603     hbck = doFsck(conf, false);
2604     assertNoErrors(hbck); // should not have expired, no problems
2605 
2606     edge.incrementTime(conf.getLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT,
2607         TableLockManager.DEFAULT_TABLE_LOCK_EXPIRE_TIMEOUT_MS)); // let table lock expire
2608 
2609     hbck = doFsck(conf, false);
2610     assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK});
2611 
2612     final CountDownLatch latch = new CountDownLatch(1);
2613     new Thread() {
2614       @Override
2615       public void run() {
2616         TableLock readLock = tableLockManager.writeLock(tableName, "testCheckTableLocks");
2617         try {
2618           latch.countDown();
2619           readLock.acquire();
2620         } catch (IOException ex) {
2621           fail();
2622         } catch (IllegalStateException ex) {
2623           return; // expected, since this will be reaped under us.
2624         }
2625         fail("should not have come here");
2626       };
2627     }.start();
2628 
2629     latch.await(); // wait until thread starts
2630     Threads.sleep(300); // wait some more to ensure writeLock.acquire() is called
2631 
2632     hbck = doFsck(conf, false);
2633     // still one expired, one not-expired
2634     assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK});
2635 
2636     edge.incrementTime(conf.getLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT,
2637         TableLockManager.DEFAULT_TABLE_LOCK_EXPIRE_TIMEOUT_MS)); // let table lock expire
2638 
2639     hbck = doFsck(conf, false);
2640     // both are expired
2641     assertErrors(
2642       hbck,
2643       new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK, ERROR_CODE.EXPIRED_TABLE_LOCK});
2644 
2645     Configuration localConf = new Configuration(conf);
2646     // reaping from ZKInterProcessWriteLock uses znode cTime,
2647     // which is not injectable through EnvironmentEdge
2648     localConf.setLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT, 1);
2649     Threads.sleep(10);
2650     hbck = doFsck(localConf, true); // now fix both cases
2651 
2652     hbck = doFsck(localConf, false);
2653     assertNoErrors(hbck);
2654 
2655     // ensure that locks are deleted
2656     writeLock = tableLockManager.writeLock(tableName, "should acquire without blocking");
2657     writeLock.acquire(); // this should not block.
2658     writeLock.release(); // release for clean state
2659     tableLockManager.tableDeleted(tableName);
2660   }
2661 
2662   /**
2663    * Test orphaned table ZNode (for table states)
2664    */
2665   @Test
2666   public void testOrphanedTableZNode() throws Exception {
2667     TableName table = TableName.valueOf("testOrphanedZKTableEntry");
2668 
2669     try {
2670       TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getTableStateManager()
2671       .setTableState(table, ZooKeeperProtos.Table.State.ENABLING);
2672 
2673       try {
2674         setupTable(table);
2675         Assert.fail(
2676           "Create table should fail when its ZNode has already existed with ENABLING state.");
2677       } catch(TableExistsException t) {
2678         //Expected exception
2679       }
2680       // The setup table was interrupted in some state that needs to some cleanup.
2681       try {
2682         cleanupTable(table);
2683       } catch (IOException e) {
2684         // Because create table failed, it is expected that the cleanup table would
2685         // throw some exception.  Ignore and continue.
2686       }
2687 
2688       HBaseFsck hbck = doFsck(conf, false);
2689       assertTrue(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY));
2690 
2691       // fix the orphaned ZK entry
2692       hbck = doFsck(conf, true);
2693 
2694       // check that orpahned ZK table entry is gone.
2695       hbck = doFsck(conf, false);
2696       assertFalse(hbck.getErrors().getErrorList().contains(ERROR_CODE.ORPHANED_ZK_TABLE_ENTRY));
2697       // Now create table should succeed.
2698       setupTable(table);
2699     } finally {
2700       // This code could be called that either a table was created successfully or set up
2701       // table failed in some unknown state.  Therefore, clean up can either succeed or fail.
2702       try {
2703         cleanupTable(table);
2704       } catch (IOException e) {
2705         // The cleanup table would throw some exception if create table failed in some state.
2706         // Ignore this exception
2707       }
2708     }
2709   }
2710 
2711   @Test (timeout=180000)
2712   public void testMetaOffline() throws Exception {
2713     // check no errors
2714     HBaseFsck hbck = doFsck(conf, false);
2715     assertNoErrors(hbck);
2716     deleteMetaRegion(conf, true, false, false);
2717     hbck = doFsck(conf, false);
2718     // ERROR_CODE.UNKNOWN is coming because we reportError with a message for the hbase:meta
2719     // inconsistency and whether we will be fixing it or not.
2720     assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_META_REGION, ERROR_CODE.UNKNOWN });
2721     hbck = doFsck(conf, true);
2722     assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_META_REGION, ERROR_CODE.UNKNOWN });
2723     hbck = doFsck(conf, false);
2724     assertNoErrors(hbck);
2725   }
2726 
2727   private void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs,
2728       boolean regionInfoOnly) throws IOException, InterruptedException {
2729     HRegionLocation metaLocation = connection.getRegionLocator(TableName.META_TABLE_NAME)
2730         .getRegionLocation(HConstants.EMPTY_START_ROW);
2731     ServerName hsa = metaLocation.getServerName();
2732     HRegionInfo hri = metaLocation.getRegionInfo();
2733     if (unassign) {
2734       LOG.info("Undeploying meta region " + hri + " from server " + hsa);
2735       try (Connection unmanagedConnection = ConnectionFactory.createConnection(conf)) {
2736         undeployRegion(unmanagedConnection, hsa, hri);
2737       }
2738     }
2739 
2740     if (regionInfoOnly) {
2741       LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString());
2742       Path rootDir = FSUtils.getRootDir(conf);
2743       FileSystem fs = rootDir.getFileSystem(conf);
2744       Path p = new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(),
2745           hri.getEncodedName());
2746       Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE);
2747       fs.delete(hriPath, true);
2748     }
2749 
2750     if (hdfs) {
2751       LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString());
2752       Path rootDir = FSUtils.getRootDir(conf);
2753       FileSystem fs = rootDir.getFileSystem(conf);
2754       Path p = new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(),
2755           hri.getEncodedName());
2756       HBaseFsck.debugLsr(conf, p);
2757       boolean success = fs.delete(p, true);
2758       LOG.info("Deleted " + p + " sucessfully? " + success);
2759       HBaseFsck.debugLsr(conf, p);
2760     }
2761   }
2762 
2763   @Test (timeout=180000)
2764   public void testTableWithNoRegions() throws Exception {
2765     // We might end up with empty regions in a table
2766     // see also testNoHdfsTable()
2767     TableName table =
2768         TableName.valueOf(name.getMethodName());
2769     try {
2770       // create table with one region
2771       HTableDescriptor desc = new HTableDescriptor(table);
2772       HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
2773       desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
2774       createTable(TEST_UTIL, desc, null);
2775       tbl = (HTable) connection.getTable(table, tableExecutorService);
2776 
2777       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
2778       deleteRegion(conf, tbl.getTableDescriptor(), HConstants.EMPTY_START_ROW,
2779           HConstants.EMPTY_END_ROW, false, false, true);
2780 
2781       HBaseFsck hbck = doFsck(conf, false);
2782       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
2783 
2784       doFsck(conf, true);
2785 
2786       // fix hole
2787       doFsck(conf, true);
2788 
2789       // check that hole fixed
2790       assertNoErrors(doFsck(conf, false));
2791     } finally {
2792       cleanupTable(table);
2793     }
2794 
2795   }
2796 
2797   @Test (timeout=180000)
2798   public void testHbckAfterRegionMerge() throws Exception {
2799     TableName table = TableName.valueOf("testMergeRegionFilesInHdfs");
2800     Table meta = null;
2801     try {
2802       // disable CatalogJanitor
2803       TEST_UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
2804       setupTable(table);
2805       assertEquals(ROWKEYS.length, countRows());
2806 
2807       // make sure data in regions, if in wal only there is no data loss
2808       admin.flush(table);
2809       HRegionInfo region1 = tbl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
2810       HRegionInfo region2 = tbl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
2811 
2812       int regionCountBeforeMerge = tbl.getRegionLocations().size();
2813 
2814       assertNotEquals(region1, region2);
2815 
2816       // do a region merge
2817       admin.mergeRegions(region1.getEncodedNameAsBytes(),
2818           region2.getEncodedNameAsBytes(), false);
2819 
2820       // wait until region merged
2821       long timeout = System.currentTimeMillis() + 30 * 1000;
2822       while (true) {
2823         if (tbl.getRegionLocations().size() < regionCountBeforeMerge) {
2824           break;
2825         } else if (System.currentTimeMillis() > timeout) {
2826           fail("Time out waiting on region " + region1.getEncodedName()
2827               + " and " + region2.getEncodedName() + " be merged");
2828         }
2829         Thread.sleep(10);
2830       }
2831 
2832       assertEquals(ROWKEYS.length, countRows());
2833 
2834       HBaseFsck hbck = doFsck(conf, false);
2835       assertNoErrors(hbck); // no errors
2836 
2837     } finally {
2838       TEST_UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
2839       cleanupTable(table);
2840       IOUtils.closeQuietly(meta);
2841     }
2842   }
2843 
2844   @Test (timeout = 180000)
2845   public void testRegionBoundariesCheck() throws Exception {
2846     HBaseFsck hbck = doFsck(conf, false);
2847     assertNoErrors(hbck); // no errors
2848     try {
2849       hbck.checkRegionBoundaries();
2850     } catch (IllegalArgumentException e) {
2851       if (e.getMessage().endsWith("not a valid DFS filename.")) {
2852         fail("Table directory path is not valid." + e.getMessage());
2853       }
2854     }
2855   }
2856 
2857   @org.junit.Rule
2858   public TestName name = new TestName();
2859 
2860   @Test (timeout=180000)
2861   public void testReadOnlyProperty() throws Exception {
2862     HBaseFsck hbck = doFsck(conf, false);
2863     Assert.assertEquals("shouldIgnorePreCheckPermission", true,
2864       hbck.shouldIgnorePreCheckPermission());
2865 
2866     hbck = doFsck(conf, true);
2867     Assert.assertEquals("shouldIgnorePreCheckPermission", false,
2868       hbck.shouldIgnorePreCheckPermission());
2869 
2870     hbck = doFsck(conf, true);
2871     hbck.setIgnorePreCheckPermission(true);
2872     Assert.assertEquals("shouldIgnorePreCheckPermission", true,
2873       hbck.shouldIgnorePreCheckPermission());
2874   }
2875 
2876   @Test (timeout=180000)
2877   public void testCleanUpDaughtersNotInMetaAfterFailedSplit() throws Exception {
2878     TableName table = TableName.valueOf("testCleanUpDaughtersNotInMetaAfterFailedSplit");
2879     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
2880     try {
2881       HTableDescriptor desc = new HTableDescriptor(table);
2882       desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
2883       createTable(TEST_UTIL, desc, null);
2884       tbl = new HTable(cluster.getConfiguration(), desc.getTableName());
2885       for (int i = 0; i < 5; i++) {
2886         Put p1 = new Put(("r" + i).getBytes());
2887         p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
2888         tbl.put(p1);
2889       }
2890       admin.flush(desc.getTableName());
2891       List<HRegion> regions = cluster.getRegions(desc.getTableName());
2892       int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
2893       HRegionServer regionServer = cluster.getRegionServer(serverWith);
2894       cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
2895       SplitTransactionImpl st = new SplitTransactionImpl(regions.get(0), Bytes.toBytes("r3"));
2896       st.prepare();
2897       st.stepsBeforePONR(regionServer, regionServer, false);
2898       AssignmentManager am = cluster.getMaster().getAssignmentManager();
2899       Map<String, RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
2900       for (RegionState state : regionsInTransition.values()) {
2901         am.regionOffline(state.getRegion());
2902       }
2903       ZKAssign.deleteNodeFailSilent(regionServer.getZooKeeper(), regions.get(0).getRegionInfo());
2904       Map<HRegionInfo, ServerName> regionsMap = new HashMap<HRegionInfo, ServerName>();
2905       regionsMap.put(regions.get(0).getRegionInfo(), regionServer.getServerName());
2906       am.assign(regionsMap);
2907       am.waitForAssignment(regions.get(0).getRegionInfo());
2908       HBaseFsck hbck = doFsck(conf, false);
2909       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
2910           ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
2911       // holes are separate from overlap groups
2912       assertEquals(0, hbck.getOverlapGroups(table).size());
2913 
2914       // fix hole
2915       assertErrors(
2916         doFsck(
2917           conf, false, true, false, false, false, false, false, false, false, false, false, null),
2918         new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
2919           ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
2920 
2921       // check that hole fixed
2922       assertNoErrors(doFsck(conf, false));
2923       assertEquals(5, countRows());
2924     } finally {
2925       if (tbl != null) {
2926         tbl.close();
2927         tbl = null;
2928       }
2929       cleanupTable(table);
2930     }
2931   }
2932 
2933 
2934   public static class MasterSyncObserver extends BaseMasterObserver {
2935     volatile CountDownLatch tableCreationLatch = null;
2936     volatile CountDownLatch tableDeletionLatch = null;
2937 
2938     @Override
2939     public void postCreateTableHandler(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2940       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
2941       // the AccessController test, some times calls only and directly the postCreateTableHandler()
2942       if (tableCreationLatch != null) {
2943         tableCreationLatch.countDown();
2944       }
2945     }
2946 
2947     @Override
2948     public void postDeleteTableHandler(final ObserverContext<MasterCoprocessorEnvironment> ctx,
2949                                        TableName tableName)
2950     throws IOException {
2951       // the AccessController test, some times calls only and directly the postDeleteTableHandler()
2952       if (tableDeletionLatch != null) {
2953         tableDeletionLatch.countDown();
2954       }
2955     }
2956   }
2957 
2958   public static void createTable(HBaseTestingUtility testUtil, HTableDescriptor htd,
2959     byte [][] splitKeys) throws Exception {
2960     // NOTE: We need a latch because admin is not sync,
2961     // so the postOp coprocessor method may be called after the admin operation returned.
2962     MasterSyncObserver observer = (MasterSyncObserver)testUtil.getHBaseCluster().getMaster()
2963       .getMasterCoprocessorHost().findCoprocessor(MasterSyncObserver.class.getName());
2964     observer.tableCreationLatch = new CountDownLatch(1);
2965     if (splitKeys != null) {
2966       admin.createTable(htd, splitKeys);
2967     } else {
2968       admin.createTable(htd);
2969     }
2970     observer.tableCreationLatch.await();
2971     observer.tableCreationLatch = null;
2972     testUtil.waitUntilAllRegionsAssigned(htd.getTableName());
2973   }
2974 
2975   public static void deleteTable(HBaseTestingUtility testUtil, TableName tableName)
2976     throws Exception {
2977     // NOTE: We need a latch because admin is not sync,
2978     // so the postOp coprocessor method may be called after the admin operation returned.
2979     MasterSyncObserver observer = (MasterSyncObserver)testUtil.getHBaseCluster().getMaster()
2980       .getMasterCoprocessorHost().findCoprocessor(MasterSyncObserver.class.getName());
2981     observer.tableDeletionLatch = new CountDownLatch(1);
2982     try {
2983       admin.disableTable(tableName);
2984     } catch (Exception e) {
2985       LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
2986     }
2987     admin.deleteTable(tableName);
2988     observer.tableDeletionLatch.await();
2989     observer.tableDeletionLatch = null;
2990   }
2991 }