You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/05/14 22:16:03 UTC

[hbase] branch branch-1.4 updated: HBASE-22391 Fix flaky tests from TestFromClientSide

This is an automated email from the ASF dual-hosted git repository.

apurtell pushed a commit to branch branch-1.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1.4 by this push:
     new a151bc0  HBASE-22391 Fix flaky tests from TestFromClientSide
a151bc0 is described below

commit a151bc03b703f95923c598f2058fe035146da0f6
Author: Xu Cang <xu...@apache.org>
AuthorDate: Thu May 9 17:04:20 2019 -0700

    HBASE-22391 Fix flaky tests from TestFromClientSide
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/client/TestFromClientSide.java    | 160 ++++++++++++++++-----
 1 file changed, 125 insertions(+), 35 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 254d8f2..228c1c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -193,7 +193,7 @@ public class TestFromClientSide {
     kvs.put(HConnectionTestingUtility.SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
     hdt.addCoprocessor(HConnectionTestingUtility.SleepAtFirstRpcCall.class.getName(), null, 1, kvs);
     TEST_UTIL.createTable(hdt, new byte[][] { ROW }).close();
-
+    TEST_UTIL.waitTableAvailable(hdt.getTableName(), 10000);
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
     // Client will retry beacuse rpc timeout is small than the sleep time of first rpc call
@@ -245,6 +245,7 @@ public class TestFromClientSide {
      HTableDescriptor desc = new HTableDescriptor(TABLENAME);
      desc.addFamily(hcd);
      TEST_UTIL.getHBaseAdmin().createTable(desc);
+     TEST_UTIL.waitTableAvailable(desc.getTableName(), 10000);
      Configuration c = TEST_UTIL.getConfiguration();
      Table h = new HTable(c, TABLENAME);
 
@@ -307,7 +308,7 @@ public class TestFromClientSide {
      final byte[] VALUE = Bytes.toBytes("value");
 
      Table table = TEST_UTIL.createTable(TABLENAME, FAMILY);
-
+     TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
      // future timestamp
      long ts = System.currentTimeMillis() * 2;
      Put put = new Put(ROW, ts);
@@ -450,6 +451,7 @@ public class TestFromClientSide {
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     Configuration conf = TEST_UTIL.getConfiguration();
     Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     assertSame(conf, table.getConfiguration());
   }
 
@@ -466,6 +468,7 @@ public class TestFromClientSide {
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
     HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     String value = "this is the value";
     String value2 = "this is some other value";
     String keyPrefix1 = UUID.randomUUID().toString();
@@ -596,6 +599,7 @@ public class TestFromClientSide {
   throws IOException, InterruptedException {
     TableName name = TableName.valueOf("testFilterAcrossMutlipleRegions");
     HTable t = TEST_UTIL.createTable(name, FAMILY);
+    TEST_UTIL.waitTableAvailable(name, 10000);
     int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
     assertRowCount(t, rowCount);
     // Split the table.  Should split on a reasonable key; 'lqj'
@@ -714,6 +718,7 @@ public class TestFromClientSide {
     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
     TEST_UTIL.getHBaseAdmin().createTable(htd);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
     ExceptionInReseekRegionObserver.reset();
     ExceptionInReseekRegionObserver.throwOnce.set(true); // throw exceptions only once
     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
@@ -738,6 +743,7 @@ public class TestFromClientSide {
     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
     TEST_UTIL.getHBaseAdmin().createTable(htd);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
     ExceptionInReseekRegionObserver.reset();
     ExceptionInReseekRegionObserver.isDoNotRetry.set(true);
     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
@@ -765,6 +771,7 @@ public class TestFromClientSide {
     HTableDescriptor htd = TEST_UTIL.createTableDescriptor(name, FAMILY);
     htd.addCoprocessor(ExceptionInReseekRegionObserver.class.getName());
     TEST_UTIL.getHBaseAdmin().createTable(htd);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
     ExceptionInReseekRegionObserver.reset();
     ExceptionInReseekRegionObserver.throwOnce.set(false); // throw exceptions in every retry
     try (Table t = TEST_UTIL.getConnection().getTable(name)) {
@@ -874,6 +881,7 @@ public class TestFromClientSide {
   public void testSuperSimple() throws Exception {
     byte [] TABLE = Bytes.toBytes("testSuperSimple");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
     ht.put(put);
@@ -891,6 +899,7 @@ public class TestFromClientSide {
     Configuration conf = TEST_UTIL.getConfiguration();
     String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte[] value = new byte[4 * 1024 * 1024];
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, value);
@@ -916,6 +925,7 @@ public class TestFromClientSide {
   public void testFilters() throws Exception {
     byte [] TABLE = Bytes.toBytes("testFilters");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] QUALIFIERS = {
         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@@ -952,6 +962,7 @@ public class TestFromClientSide {
   public void testFilterWithLongCompartor() throws Exception {
     byte [] TABLE = Bytes.toBytes("testFilterWithLongCompartor");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] values = new byte[10][];
     for (int i = 0; i < 10; i ++) {
@@ -983,6 +994,7 @@ public class TestFromClientSide {
   public void testKeyOnlyFilter() throws Exception {
     byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] QUALIFIERS = {
         Bytes.toBytes("col0-<d2v1>-<d3v2>"), Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@@ -1020,6 +1032,7 @@ public class TestFromClientSide {
   public void testSimpleMissing() throws Exception {
     byte [] TABLE = Bytes.toBytes("testSimpleMissing");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte [][] ROWS = makeN(ROW, 4);
 
     // Try to get a row on an empty table
@@ -1136,6 +1149,7 @@ public class TestFromClientSide {
     byte [][] VALUES = makeN(VALUE, 10);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     Get get;
     Scan scan;
@@ -1445,6 +1459,7 @@ public class TestFromClientSide {
     } catch(Exception e) {}
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Null row (should NOT work)
     try {
@@ -1476,7 +1491,7 @@ public class TestFromClientSide {
     // Use a new table
     byte [] TABLE2 = Bytes.toBytes("testNull2");
     ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY);
-
+    TEST_UTIL.waitTableAvailable(TableName.valueOf(TABLE2), 10000);
     // Empty qualifier, byte[0] instead of null (should work)
     try {
       Put put = new Put(ROW);
@@ -1544,7 +1559,7 @@ public class TestFromClientSide {
     byte [][] VALUES = makeNAscii(VALUE, 20);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
-
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     // Insert 4 versions of same column
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
@@ -1758,7 +1773,7 @@ public class TestFromClientSide {
     long [] STAMPS = makeStamps(10);
     byte [][] VALUES = makeNAscii(VALUE, 10);
     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, LIMITS);
-
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     // Insert limit + 1 on each family
     Put put = new Put(ROW);
     put.add(FAMILIES[0], QUALIFIER, STAMPS[0], VALUES[0]);
@@ -1954,6 +1969,7 @@ public class TestFromClientSide {
     long [] ts = {1000, 2000, 3000, 4000, 5000};
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     Put put = new Put(ROW);
     for (int q = 0; q < 1; q++)
@@ -1993,6 +2009,8 @@ public class TestFromClientSide {
 
     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 5);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     Put put = null;
     Result result = null;
     Get get = null;
@@ -2108,6 +2126,7 @@ public class TestFromClientSide {
     long [] ts = {1000, 2000, 3000, 4000, 5000};
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     Put put = new Put(ROW);
     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
@@ -2475,6 +2494,7 @@ public class TestFromClientSide {
     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert rows
 
@@ -2558,6 +2578,7 @@ public class TestFromClientSide {
     long [] STAMPS = makeStamps(7);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert three versions
 
@@ -2622,6 +2643,7 @@ public class TestFromClientSide {
     long [] STAMPS = makeStamps(7);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert lots versions
 
@@ -2669,6 +2691,7 @@ public class TestFromClientSide {
     byte [] TABLE = Bytes.toBytes("testJiraTest1014");
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     long manualStamp = 12345;
 
@@ -2696,6 +2719,7 @@ public class TestFromClientSide {
     long [] STAMPS = makeStamps(7);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert lots versions
 
@@ -2739,6 +2763,7 @@ public class TestFromClientSide {
     long [] STAMPS = makeStamps(7);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert lots versions
 
@@ -3570,6 +3595,7 @@ public class TestFromClientSide {
     byte [][] VALUES = makeNAscii(VALUE, 20);
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert 4 versions of same column
     Put put = new Put(ROW);
@@ -3781,6 +3807,7 @@ public class TestFromClientSide {
 
     byte [] TABLE = Bytes.toBytes("testUpdates");
     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Write a column with values at timestamp 1, 2 and 3
     byte[] row = Bytes.toBytes("row1");
@@ -3831,6 +3858,8 @@ public class TestFromClientSide {
 
     TableName TABLE = TableName.valueOf("testUpdatesWithMajorCompaction");
     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 
     // Write a column with values at timestamp 1, 2 and 3
@@ -3893,6 +3922,7 @@ public class TestFromClientSide {
     String tableName = "testMajorCompactionBetweenTwoUpdates";
     byte [] TABLE = Bytes.toBytes(tableName);
     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
 
     // Write a column with values at timestamp 1, 2 and 3
@@ -3956,8 +3986,9 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testGet_EmptyTable() throws IOException {
+  public void testGet_EmptyTable() throws IOException, InterruptedException {
     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_EmptyTable"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_EmptyTable"), 10000);
     Get get = new Get(ROW);
     get.addFamily(FAMILY);
     Result r = table.get(get);
@@ -3965,8 +3996,9 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testGet_NullQualifier() throws IOException {
+  public void testGet_NullQualifier() throws IOException, InterruptedException {
     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NullQualifier"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_NullQualifier"), 10000);
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
     table.put(put);
@@ -3988,8 +4020,9 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testGet_NonExistentRow() throws IOException {
+  public void testGet_NonExistentRow() throws IOException, InterruptedException {
     Table table = TEST_UTIL.createTable(TableName.valueOf("testGet_NonExistentRow"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testGet_NonExistentRow"), 10000);
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
     table.put(put);
@@ -4010,7 +4043,7 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testPut() throws IOException {
+  public void testPut() throws IOException, InterruptedException {
     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte [] row1 = Bytes.toBytes("row1");
@@ -4018,6 +4051,7 @@ public class TestFromClientSide {
     final byte [] value = Bytes.toBytes("abcd");
     Table table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testPut"), 10000);
     Put put = new Put(row1);
     put.add(CONTENTS_FAMILY, null, value);
     table.put(put);
@@ -4050,10 +4084,11 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testPutNoCF() throws IOException {
+  public void testPutNoCF() throws IOException, InterruptedException {
     final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
     final byte[] VAL = Bytes.toBytes(100);
     Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), FAMILY);
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testPutNoCF"), 10000);
 
     boolean caughtNSCFE = false;
 
@@ -4069,13 +4104,14 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testRowsPut() throws IOException {
+  public void testRowsPut() throws IOException, InterruptedException {
     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final int NB_BATCH_ROWS = 10;
     final byte[] value = Bytes.toBytes("abcd");
     Table table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPut"), 10000);
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS; i++) {
       byte[] row = Bytes.toBytes("row" + i);
@@ -4096,13 +4132,15 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testRowsPutBufferedOneFlush() throws IOException {
+  public void testRowsPutBufferedOneFlush() throws IOException, InterruptedException {
     final byte [] CONTENTS_FAMILY = Bytes.toBytes("contents");
     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte [] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
       new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
+        10000);
     table.setAutoFlush(false);
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
@@ -4138,13 +4176,15 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testRowsPutBufferedManyManyFlushes() throws IOException {
+  public void testRowsPutBufferedManyManyFlushes() throws IOException, InterruptedException {
     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte[] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
     HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
+        10000);
     table.setWriteBufferSize(10);
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
@@ -4196,10 +4236,11 @@ public class TestFromClientSide {
     assertEquals(true, ok);
   }
 
-  private List<Result> doAppend(final boolean walUsed) throws IOException {
+  private List<Result> doAppend(final boolean walUsed) throws IOException, InterruptedException {
     LOG.info("Starting testAppend, walUsed is " + walUsed);
     final TableName TABLENAME = TableName.valueOf(walUsed ? "testAppendWithWAL" : "testAppendWithoutWAL");
     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
     final byte[] row1 = Bytes.toBytes("c");
     final byte[] row2 = Bytes.toBytes("b");
     final byte[] row3 = Bytes.toBytes("a");
@@ -4262,12 +4303,13 @@ public class TestFromClientSide {
    * @throws IOException
    */
   @Test
-  public void testHBase737 () throws IOException {
+  public void testHBase737() throws IOException, InterruptedException {
     final byte [] FAM1 = Bytes.toBytes("fam1");
     final byte [] FAM2 = Bytes.toBytes("fam2");
     // Open table
     Table table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
       new byte [][] {FAM1, FAM2});
+    TEST_UTIL.waitTableAvailable(Bytes.toBytes("testHBase737"), 10000);
     // Insert some values
     Put put = new Put(ROW);
     put.add(FAM1, Bytes.toBytes("letters"), Bytes.toBytes("abcdefg"));
@@ -4360,6 +4402,7 @@ public class TestFromClientSide {
     TableName [] tables = new TableName[] { t1, t2, t3 };
     for (int i = 0; i < tables.length; i++) {
       TEST_UTIL.createTable(tables[i], FAMILY);
+      TEST_UTIL.waitTableAvailable(tables[i], 10000);
     }
     Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
     HTableDescriptor[] ts = admin.listTables();
@@ -4386,8 +4429,10 @@ public class TestFromClientSide {
    * @return the created HTable object
    * @throws IOException
    */
-  HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException {
+  HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException,
+      InterruptedException {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
+    TEST_UTIL.waitTableAvailable(tableName, 10000);
     HConnection conn = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
     return (HTable)conn.getTable(tableName);
   }
@@ -4399,7 +4444,7 @@ public class TestFromClientSide {
    * @throws IOException
    */
   @Test
-  public void testUnmanagedHConnection() throws IOException {
+  public void testUnmanagedHConnection() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf("testUnmanagedHConnection");
     HTable t = createUnmangedHConnectionHTable(tableName);
     HBaseAdmin ha = new HBaseAdmin(t.getConnection());
@@ -4442,7 +4487,7 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testMiscHTableStuff() throws IOException {
+  public void testMiscHTableStuff() throws IOException, InterruptedException {
     final TableName tableAname = TableName.valueOf("testMiscHTableStuffA");
     final TableName tableBname = TableName.valueOf("testMiscHTableStuffB");
     final byte[] attrName = Bytes.toBytes("TESTATTR");
@@ -4450,7 +4495,9 @@ public class TestFromClientSide {
     byte[] value = Bytes.toBytes("value");
 
     Table a = TEST_UTIL.createTable(tableAname, HConstants.CATALOG_FAMILY);
+    TEST_UTIL.waitTableAvailable(tableAname, 10000);
     Table b = TEST_UTIL.createTable(tableBname, HConstants.CATALOG_FAMILY);
+    TEST_UTIL.waitTableAvailable(tableBname, 10000);
     Put put = new Put(ROW);
     put.add(HConstants.CATALOG_FAMILY, null, value);
     a.put(put);
@@ -4532,7 +4579,9 @@ public class TestFromClientSide {
 
     HTable table =
         TEST_UTIL.createTable(tableAname,
-          new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024);
+          new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1,
+            1024);
+    TEST_UTIL.waitTableAvailable(tableAname, 10000);
     // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
     // in Store.rowAtOrBeforeFromStoreFile
     String regionName = table.getRegionLocations().firstKey().getEncodedName();
@@ -4639,6 +4688,7 @@ public class TestFromClientSide {
     final byte [] ROW1 = Bytes.toBytes("testRow1");
 
     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
     Put p = new Put(ROW);
     p.add(FAMILY, QUALIFIER, VALUE);
     MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
@@ -4668,6 +4718,7 @@ public class TestFromClientSide {
     LOG.info("Starting testRowMutation");
     final TableName TABLENAME = TableName.valueOf("testRowMutation");
     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
     byte [][] QUALIFIERS = new byte [][] {
         Bytes.toBytes("a"), Bytes.toBytes("b")
     };
@@ -4717,6 +4768,7 @@ public class TestFromClientSide {
     LOG.info("Starting testAppend");
     final TableName TABLENAME = TableName.valueOf("testAppend");
     Table t = TEST_UTIL.createTable(TABLENAME, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLENAME, 10000);
     byte[] v1 = Bytes.toBytes("42");
     byte[] v2 = Bytes.toBytes("23");
     byte [][] QUALIFIERS = new byte [][] {
@@ -4742,7 +4794,7 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testClientPoolRoundRobin() throws IOException {
+  public void testClientPoolRoundRobin() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf("testClientPoolRoundRobin");
 
     int poolSize = 3;
@@ -4752,6 +4804,7 @@ public class TestFromClientSide {
     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
 
     Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE);
+    TEST_UTIL.waitTableAvailable(tableName, 10000);
 
     final long ts = EnvironmentEdgeManager.currentTime();
     Get get = new Get(ROW);
@@ -4778,7 +4831,7 @@ public class TestFromClientSide {
   }
 
   @Ignore ("Flakey: HBASE-8989") @Test
-  public void testClientPoolThreadLocal() throws IOException {
+  public void testClientPoolThreadLocal() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf("testClientPoolThreadLocal");
 
     int poolSize = Integer.MAX_VALUE;
@@ -4789,7 +4842,7 @@ public class TestFromClientSide {
 
     final Table table = TEST_UTIL.createTable(tableName,
         new byte[][] { FAMILY }, conf, 3);
-
+    TEST_UTIL.waitTableAvailable(tableName, 10000);
     final long ts = EnvironmentEdgeManager.currentTime();
     final Get get = new Get(ROW);
     get.addColumn(FAMILY, QUALIFIER);
@@ -4861,11 +4914,12 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testCheckAndPut() throws IOException {
+  public void testCheckAndPut() throws IOException, InterruptedException {
     final byte [] anotherrow = Bytes.toBytes("anotherrow");
     final byte [] value2 = Bytes.toBytes("abcd");
 
     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPut"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndPut"), 10000);
     Put put1 = new Put(ROW);
     put1.add(FAMILY, QUALIFIER, VALUE);
 
@@ -4896,17 +4950,17 @@ public class TestFromClientSide {
         ok = table.checkAndPut(ROW, FAMILY, QUALIFIER, value2, put3);
         fail("trying to check and modify different rows should have failed.");
     } catch(Exception e) {}
-
   }
 
   @Test
-  public void testCheckAndPutWithCompareOp() throws IOException {
+  public void testCheckAndPutWithCompareOp() throws IOException, InterruptedException {
     final byte [] value1 = Bytes.toBytes("aaaa");
     final byte [] value2 = Bytes.toBytes("bbbb");
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
 
     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndPutWithCompareOp"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndPutWithCompareOp"), 10000);
 
     Put put2 = new Put(ROW);
     put2.add(FAMILY, QUALIFIER, value2);
@@ -4965,14 +5019,14 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testCheckAndDeleteWithCompareOp() throws IOException {
+  public void testCheckAndDeleteWithCompareOp() throws IOException, InterruptedException {
     final byte [] value1 = Bytes.toBytes("aaaa");
     final byte [] value2 = Bytes.toBytes("bbbb");
     final byte [] value3 = Bytes.toBytes("cccc");
     final byte [] value4 = Bytes.toBytes("dddd");
-
     Table table = TEST_UTIL.createTable(TableName.valueOf("testCheckAndDeleteWithCompareOp"),
         FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testCheckAndDeleteWithCompareOp"), 10000);
 
     Put put2 = new Put(ROW);
     put2.add(FAMILY, QUALIFIER, value2);
@@ -4998,25 +5052,30 @@ public class TestFromClientSide {
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value1, delete);
     assertEquals(ok, true);
     table.put(put2);
-    ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value1, delete);
+
     assertEquals(ok, true);
 
     // cell = "cccc", using "dddd" to compare only LARGER/LARGER_OR_EQUAL/NOT_EQUAL
     // turns out "match"
     table.put(put3);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS, value4, delete);
+
     assertEquals(ok, false);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.LESS_OR_EQUAL, value4, delete);
+
     assertEquals(ok, false);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.EQUAL, value4, delete);
+
     assertEquals(ok, false);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER, value4, delete);
+
     assertEquals(ok, true);
     table.put(put3);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.GREATER_OR_EQUAL, value4, delete);
     assertEquals(ok, true);
     table.put(put3);
     ok = table.checkAndDelete(ROW, FAMILY, QUALIFIER, CompareOp.NOT_EQUAL, value4, delete);
+
     assertEquals(ok, true);
 
     // cell = "bbbb", using "bbbb" to compare only GREATER_OR_EQUAL/LESS_OR_EQUAL/EQUAL
@@ -5188,6 +5247,7 @@ public class TestFromClientSide {
     TableName tableName = TableName.valueOf("testCOWEOCfromClient");
     byte [] data = Bytes.toBytes("data");
     HTable table = TEST_UTIL.createTable(tableName, FAMILY);
+    TEST_UTIL.waitTableAvailable(tableName, 10000);
     // get the block cache and region
     String regionName = table.getRegionLocations().firstKey().getEncodedName();
     Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
@@ -5304,6 +5364,7 @@ public class TestFromClientSide {
     byte [] family2 = Bytes.toBytes("f2");
     try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
         Admin admin = new HBaseAdmin(TEST_UTIL.getConfiguration())) {
+      TEST_UTIL.waitTableAvailable(TABLE, 10000);
       Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
       assertEquals(1, regionsMap.size());
       HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
@@ -5404,7 +5465,7 @@ public class TestFromClientSide {
   public void testJira6912() throws Exception {
     TableName TABLE = TableName.valueOf("testJira6912");
     Table foo = TEST_UTIL.createTable(TABLE, new byte[][] {FAMILY}, 10);
-
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     List<Put> puts = new ArrayList<Put>();
     for (int i=0;i !=100; i++){
       Put put = new Put(Bytes.toBytes(i));
@@ -5427,8 +5488,10 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testScan_NullQualifier() throws IOException {
+  public void testScan_NullQualifier() throws IOException, InterruptedException {
     Table table = TEST_UTIL.createTable(TableName.valueOf("testScan_NullQualifier"), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testScan_NullQualifier"), 10000);
+
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
     table.put(put);
@@ -5456,9 +5519,9 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testNegativeTimestamp() throws IOException {
+  public void testNegativeTimestamp() throws IOException, InterruptedException {
     Table table = TEST_UTIL.createTable(TableName.valueOf("testNegativeTimestamp"), FAMILY);
-
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("testNegativeTimestamp"), 10000);
     try {
       Put put = new Put(ROW, -1);
       put.add(FAMILY, QUALIFIER, VALUE);
@@ -5667,6 +5730,7 @@ public class TestFromClientSide {
   public void testRawScanRespectsVersions() throws Exception {
     TableName TABLE = TableName.valueOf("testRawScan");
     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
     byte[] row = Bytes.toBytes("row");
 
     // put the same row 4 times, with different values
@@ -5738,6 +5802,7 @@ public class TestFromClientSide {
     // Test Initialization.
     TableName TABLE = TableName.valueOf("testEmptyFilterList");
     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert one row each region
     Put put = new Put(Bytes.toBytes("row"));
@@ -5782,6 +5847,7 @@ public class TestFromClientSide {
     // Test Initialization.
     TableName TABLE = TableName.valueOf("testSmallScan");
     Table table = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     // Insert one row each region
     int insertNum = 10;
@@ -5818,6 +5884,8 @@ public class TestFromClientSide {
   public void testSuperSimpleWithReverseScan() throws Exception {
     TableName TABLE = TableName.valueOf("testSuperSimpleWithReverseScan");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     Put put = new Put(Bytes.toBytes("0-b11111-0000000000000000000"));
     put.add(FAMILY, QUALIFIER, VALUE);
     ht.put(put);
@@ -5863,6 +5931,8 @@ public class TestFromClientSide {
   public void testFiltersWithReverseScan() throws Exception {
     TableName TABLE = TableName.valueOf("testFiltersWithReverseScan");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     byte[][] ROWS = makeN(ROW, 10);
     byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
         Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@@ -5903,6 +5973,8 @@ public class TestFromClientSide {
   public void testKeyOnlyFilterWithReverseScan() throws Exception {
     TableName TABLE = TableName.valueOf("testKeyOnlyFilterWithReverseScan");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     byte[][] ROWS = makeN(ROW, 10);
     byte[][] QUALIFIERS = { Bytes.toBytes("col0-<d2v1>-<d3v2>"),
         Bytes.toBytes("col1-<d2v1>-<d3v2>"),
@@ -5944,6 +6016,8 @@ public class TestFromClientSide {
   public void testSimpleMissingWithReverseScan() throws Exception {
     TableName TABLE = TableName.valueOf("testSimpleMissingWithReverseScan");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     byte[][] ROWS = makeN(ROW, 4);
 
     // Try to get a row on an empty table
@@ -6009,6 +6083,8 @@ public class TestFromClientSide {
   public void testNullWithReverseScan() throws Exception {
     TableName TABLE = TableName.valueOf("testNullWithReverseScan");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     // Null qualifier (should work)
     Put put = new Put(ROW);
     put.add(FAMILY, null, VALUE);
@@ -6020,6 +6096,8 @@ public class TestFromClientSide {
     // Use a new table
     byte[] TABLE2 = Bytes.toBytes("testNull2WithReverseScan");
     ht = TEST_UTIL.createTable(TableName.valueOf(TABLE2), FAMILY);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf(TABLE2), 10000);
+
     // Empty qualifier, byte[0] instead of null (should work)
     put = new Put(ROW);
     put.add(FAMILY, HConstants.EMPTY_BYTE_ARRAY, VALUE);
@@ -6050,6 +6128,7 @@ public class TestFromClientSide {
     byte[][] VALUES = makeN(VALUE, 5);
     long[] ts = { 1000, 2000, 3000, 4000, 5000 };
     Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
 
     Put put = new Put(ROW);
     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
@@ -6240,6 +6319,8 @@ public class TestFromClientSide {
         Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)),
         Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) };
     HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
 
     assertEquals(splitRows.length + 1, table.getRegionLocations().size());
@@ -6292,6 +6373,8 @@ public class TestFromClientSide {
         Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
         Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
     HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
+    TEST_UTIL.waitTableAvailable(TABLE, 10000);
+
     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
 
     assertEquals(splitRows.length + 1, table.getRegionLocations().size());
@@ -6447,13 +6530,14 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testGetStartEndKeysWithRegionReplicas() throws IOException {
+  public void testGetStartEndKeysWithRegionReplicas() throws IOException, InterruptedException {
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testGetStartEndKeys"));
     HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
     htd.addFamily(fam);
     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(htd, KEYS);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
     List<HRegionInfo> regions = admin.getTableRegions(htd.getTableName());
 
     HRegionLocator locator =
@@ -6497,13 +6581,15 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testRegionCache() throws IOException {
+  public void testRegionCache() throws IOException, InterruptedException {
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionCache"));
     HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
     htd.addFamily(fam);
     byte[][] KEYS = HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE;
     Admin admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(htd, KEYS);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
+
     HRegionLocator locator =
       (HRegionLocator) admin.getConnection().getRegionLocator(htd.getTableName());
     List<HRegionLocation> results = locator.getAllRegionLocations();
@@ -6513,7 +6599,7 @@ public class TestFromClientSide {
   }
 
   @Test
-  public void testCellSizeLimit() throws IOException {
+  public void testCellSizeLimit() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf("testCellSizeLimit");
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.setConfiguration(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); // 10K
@@ -6521,6 +6607,8 @@ public class TestFromClientSide {
     htd.addFamily(fam);
     Admin admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(htd);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 10000);
+
     // Will succeed
     try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
       t.put(new Put(ROW).add(FAMILY, QUALIFIER, Bytes.toBytes(0L)));
@@ -6563,6 +6651,8 @@ public class TestFromClientSide {
     desc.addFamily(new HColumnDescriptor("cf"));
 
     TEST_UTIL.getHBaseAdmin().createTable(desc);
+    TEST_UTIL.waitTableAvailable(desc.getTableName(), 10000);
+
     HTableDescriptor newDesc = new HTableDescriptor(desc);
     newDesc.setRegionReplication(0);