You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/03/23 15:43:06 UTC

[02/21] hbase git commit: HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
index 31eedfc..506e537 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
@@ -22,26 +22,28 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestAddColumnFamilyProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout = 60000)
   public void testAddColumnFamily() throws Exception {
@@ -61,8 +63,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf1);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf1);
 
     // Test 2: Add a column family offline
     UTIL.getAdmin().disableTable(tableName);
@@ -71,8 +72,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId2);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf2);
   }
 
   @Test(timeout=60000)
@@ -91,8 +91,7 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf2);
 
     // add the column family that exists
     long procId2 = procExec.submitProcedure(
@@ -140,11 +139,9 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = AddColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf4);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf4);
   }
 
   @Test(timeout = 60000)
@@ -164,11 +161,9 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = AddColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf5);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf5);
   }
 
   @Test(timeout = 60000)
@@ -187,10 +182,9 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
-    int numberOfSteps = 1; // failing at "pre operations"
+    int numberOfSteps = 0; // failing at "pre operations"
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf6);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf6);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
index d8221bb..8f7686a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCloneSnapshotProcedure.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertTrue;
+
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -28,12 +30,11 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.client.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -42,8 +43,6 @@ import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.assertTrue;
-
 @Category({MasterTests.class, MediumTests.class})
 public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestCloneSnapshotProcedure.class);
@@ -146,8 +145,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
       new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = CloneSnapshotState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     MasterProcedureTestingUtility.validateTableIsEnabled(
       UTIL.getHBaseCluster().getMaster(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
index 995d98d..147ee27 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateNamespaceProcedure.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.After;
@@ -191,8 +190,7 @@ public class TestCreateNamespaceProcedure {
       new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = CreateNamespaceState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate the creation of namespace
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
index c09016c..6bd88c7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -40,8 +42,12 @@ import static org.junit.Assert.assertTrue;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
-  @Rule
-  public TestName name = new TestName();
+  private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure.class);
+
+  private static final String F1 = "f1";
+  private static final String F2 = "f2";
+
+  @Rule public TestName name = new TestName();
 
   @Test(timeout=60000)
   public void testSimpleCreate() throws Exception {
@@ -61,9 +67,8 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
 
   private void testSimpleCreate(final TableName tableName, byte[][] splitKeys) throws Exception {
     HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
-      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
-    MasterProcedureTestingUtility.validateTableCreation(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+      getMasterProcedureExecutor(), tableName, splitKeys, F1, F2);
+    MasterProcedureTestingUtility.validateTableCreation(getMaster(), tableName, regions, F1, F2);
   }
 
   @Test(timeout=60000)
@@ -126,25 +131,21 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
       new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
 
     // Restart the executor and execute the step twice
-    // NOTE: the 6 (number of CreateTableState steps) is hardcoded,
-    //       so you have to look at this test at least once when you add a new step.
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, 6);
-
-    MasterProcedureTestingUtility.validateTableCreation(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    MasterProcedureTestingUtility.validateTableCreation(getMaster(), tableName, regions, F1, F2);
   }
 
   @Test(timeout=90000)
   public void testRollbackAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    testRollbackAndDoubleExecution(MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2"));
+    testRollbackAndDoubleExecution(MasterProcedureTestingUtility.createHTD(tableName, F1, F2));
   }
 
   @Test(timeout=90000)
   public void testRollbackAndDoubleExecutionOnMobTable() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
-    htd.getFamily(Bytes.toBytes("f1")).setMobEnabled(true);
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, F1, F2);
+    htd.getFamily(Bytes.toBytes(F1)).setMobEnabled(true);
     testRollbackAndDoubleExecution(htd);
   }
 
@@ -166,11 +167,24 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     TableName tableName = htd.getTableName();
-    MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
 
     // are we able to create the table after a rollback?
     resetProcExecutorTestingKillFlag();
     testSimpleCreate(tableName, splitKeys);
   }
+
+  @Test
+  public void testMRegions() throws Exception {
+    final byte[][] splitKeys = new byte[500][];
+    for (int i = 0; i < splitKeys.length; ++i) {
+      splitKeys[i] = Bytes.toBytes(String.format("%08d", i));
+    }
+
+    final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(
+      TableName.valueOf("TestMRegions"), F1, F2);
+    UTIL.getHBaseAdmin().createTableAsync(htd, splitKeys)
+      .get(10, java.util.concurrent.TimeUnit.HOURS);
+    LOG.info("TABLE CREATED");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
index c4bdc18..6096755 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
@@ -22,26 +22,27 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestDeleteColumnFamilyProcedure.class);
-
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout = 60000)
   public void testDeleteColumnFamily() throws Exception {
@@ -59,8 +60,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf1);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf1);
 
     // Test 2: delete the column family that exists offline
     UTIL.getAdmin().disableTable(tableName);
@@ -88,8 +88,7 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf2);
 
     // delete the column family that does not exist
     long procId2 = procExec.submitProcedure(
@@ -159,11 +158,9 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = DeleteColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf4);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf4);
   }
 
   @Test(timeout = 60000)
@@ -183,11 +180,9 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = DeleteColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
-      tableName, cf5);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf5);
   }
 
   @Test(timeout = 60000)
@@ -207,10 +202,10 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
 
-    int numberOfSteps = 1; // failing at pre operation
+    int numberOfSteps = 0; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     MasterProcedureTestingUtility.validateTableCreation(
-      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
+      getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
index acedf1d..5ecacb6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteNamespaceProcedure.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.After;
@@ -175,8 +174,7 @@ public class TestDeleteNamespaceProcedure {
       new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = DeleteNamespaceState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate the deletion of namespace
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
index 431e3e4..2a11544 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
@@ -34,15 +35,16 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 import static org.junit.Assert.assertTrue;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestDeleteTableProcedure.class);
-
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout=60000, expected=TableNotFoundException.class)
   public void testDeleteNotExistentTable() throws Exception {
@@ -90,8 +92,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
 
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
 
     // Second delete should fail with TableNotFound
     ProcedureInfo result = procExec.getResult(procId2);
@@ -126,8 +127,7 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
     long procId = ProcedureTestingUtility.submitAndWait(procExec,
       new DeleteTableProcedure(procExec.getEnvironment(), tableName));
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
   }
 
   @Test(timeout=60000)
@@ -149,11 +149,8 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
       new DeleteTableProcedure(procExec.getEnvironment(), tableName));
 
     // Restart the executor and execute the step twice
-    // NOTE: the 6 (number of DeleteTableState steps) is hardcoded,
-    //       so you have to look at this test at least once when you add a new step.
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, 6);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
-    MasterProcedureTestingUtility.validateTableDeletion(
-      UTIL.getHBaseCluster().getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
index d3fccbe..11c4e2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
@@ -22,12 +22,12 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -36,13 +36,15 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestDisableTableProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
-  @Rule
-  public TestName name = new TestName();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout = 60000)
   public void testDisableTable() throws Exception {
@@ -57,8 +59,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
   }
 
   @Test(timeout = 60000)
@@ -74,8 +75,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
 
     // Disable the table again - expect failure
     long procId2 = procExec.submitProcedure(new DisableTableProcedure(
@@ -107,8 +107,7 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId4);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId4);
-    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
   }
 
   @Test(timeout=60000)
@@ -128,9 +127,8 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
       new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = DisableTableState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
-    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+
+    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
index 81f2576..01f9ed8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
@@ -22,12 +22,12 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -36,13 +36,14 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestEnableTableProcedure.class);
-
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout = 60000)
   public void testEnableTable() throws Exception {
@@ -58,8 +59,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
   }
 
   @Test(timeout=60000, expected=TableNotDisabledException.class)
@@ -113,10 +113,9 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = EnableTableState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
-    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+
+    MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
   }
 
   @Test(timeout = 60000)
@@ -136,9 +135,8 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
 
-    int numberOfSteps = 1; // failing at pre operation
+    int numberOfSteps = 0; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
-    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
-      tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index af48302..d2df2bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
@@ -55,6 +54,7 @@ import org.junit.rules.TestRule;
 public class TestMasterFailoverWithProcedures {
   private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
 
+
   @ClassRule
   public static final TestRule timeout =
       CategoryBasedTimeout.forClass(TestMasterFailoverWithProcedures.class);
@@ -116,7 +116,7 @@ public class TestMasterFailoverWithProcedures {
     HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
     long procId = procExec.submitProcedure(
         new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
-    testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
+    testRecoveryAndDoubleExecution(UTIL, procId, step);
 
     MasterProcedureTestingUtility.validateTableCreation(
         UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
@@ -154,7 +154,7 @@ public class TestMasterFailoverWithProcedures {
     // Start the Delete procedure && kill the executor
     long procId = procExec.submitProcedure(
         new DeleteTableProcedure(procExec.getEnvironment(), tableName));
-    testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
+    testRecoveryAndDoubleExecution(UTIL, procId, step);
 
     MasterProcedureTestingUtility.validateTableDeletion(
         UTIL.getHBaseCluster().getMaster(), tableName);
@@ -197,7 +197,7 @@ public class TestMasterFailoverWithProcedures {
     // Start the Truncate procedure && kill the executor
     long procId = procExec.submitProcedure(
         new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
-    testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
+    testRecoveryAndDoubleExecution(UTIL, procId, step);
 
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
     UTIL.waitUntilAllRegionsAssigned(tableName);
@@ -251,7 +251,7 @@ public class TestMasterFailoverWithProcedures {
     // Start the Delete procedure && kill the executor
     long procId = procExec.submitProcedure(
         new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
-    testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
+    testRecoveryAndDoubleExecution(UTIL, procId, step);
 
     MasterProcedureTestingUtility.validateTableIsDisabled(
         UTIL.getHBaseCluster().getMaster(), tableName);
@@ -288,7 +288,7 @@ public class TestMasterFailoverWithProcedures {
     // Start the Delete procedure && kill the executor
     long procId = procExec.submitProcedure(
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
-    testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
+    testRecoveryAndDoubleExecution(UTIL, procId, step);
 
     MasterProcedureTestingUtility.validateTableIsEnabled(
         UTIL.getHBaseCluster().getMaster(), tableName);
@@ -297,16 +297,17 @@ public class TestMasterFailoverWithProcedures {
   // ==========================================================================
   //  Test Helpers
   // ==========================================================================
-  public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
-      final long procId, final int lastStepBeforeFailover, TState[] states) throws Exception {
+  public static void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
+      final long procId, final int lastStepBeforeFailover) throws Exception {
     ProcedureExecutor<MasterProcedureEnv> procExec =
         testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
     ProcedureTestingUtility.waitProcedure(procExec, procId);
 
+    final Procedure proc = procExec.getProcedure(procId);
     for (int i = 0; i < lastStepBeforeFailover; ++i) {
-      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      LOG.info("Restart "+ i +" exec state: " + proc);
       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
-      ProcedureTestingUtility.restart(procExec);
+      MasterProcedureTestingUtility.restartMasterProcedureExecutor(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
     ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
index 6e85a9f..b7bc28f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureEvents.java
@@ -109,7 +109,7 @@ public class TestMasterProcedureEvents {
     ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
 
     while (!master.isServerCrashProcessingEnabled() || !master.isInitialized() ||
-        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+        master.getAssignmentManager().getRegionStates().hasRegionsInTransition()) {
       Thread.sleep(25);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
index 184150b..f1667ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Rule;
@@ -125,8 +124,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = ModifyColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
         .getMaster(), tableName, cf3, columnDescriptor);
@@ -153,8 +151,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = ModifyColumnFamilyState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
         .getMaster(), tableName, cf4, columnDescriptor);
@@ -180,7 +177,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
-    int numberOfSteps = 1; // failing at pre operation
+    int numberOfSteps = 0; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
index 1b53d23..5cb117b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyNamespaceProcedure.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.After;
@@ -212,8 +211,7 @@ public class TestModifyNamespaceProcedure {
       new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = ModifyNamespaceState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
     // Validate

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index c5c6484..8872c63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -30,18 +31,19 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
-  @Rule
-  public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   @Test(timeout=60000)
   public void testModifyTable() throws Exception {
@@ -208,8 +210,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = ModifyTableState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate descriptor
     HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
@@ -246,8 +247,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = ModifyTableState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate descriptor
     HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
@@ -282,7 +282,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
-    int numberOfSteps = 1; // failing at pre operation
+    int numberOfSteps = 0; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // cf2 should not be present
@@ -315,7 +315,7 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
     // Restart the executor and rollback the step twice
-    int numberOfSteps = 1; // failing at pre operation
+    int numberOfSteps = 0; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // cf2 should not be present

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index f21fe0b..94c0fd8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -18,14 +18,17 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import java.util.Random;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.util.List;
+import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -43,17 +46,19 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 import static org.junit.Assert.*;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestProcedureAdmin {
   private static final Log LOG = LogFactory.getLog(TestProcedureAdmin.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+  @Rule public TestName name = new TestName();
 
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  @Rule
-  public TestName name = new TestName();
 
   private static void setupConf(Configuration conf) {
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
index 479b206..2201763 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
@@ -18,11 +18,17 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -35,7 +41,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -46,6 +51,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -55,6 +61,8 @@ import static org.junit.Assert.assertTrue;
 @Category({MasterTests.class, MediumTests.class})
 public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestRestoreSnapshotProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   protected final TableName snapshotTableName = TableName.valueOf("testRestoreSnapshot");
   protected final byte[] CF1 = Bytes.toBytes("cf1");
@@ -201,8 +209,7 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
       new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot));
 
     // Restart the executor and execute the step twice
-    int numberOfSteps = RestoreSnapshotState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     resetProcExecutorTestingKillFlag();
     validateSnapshotRestore();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
index c6968d4..b35dad6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerCrashProcedure.java
@@ -19,48 +19,58 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.TestTableName;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-/**
- * It used to first run with DLS and then DLR but HBASE-12751 broke DLR so we disabled it here.
- */
-@Category(LargeTests.class)
-@RunWith(Parameterized.class)
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+
+@Category({MasterTests.class, LargeTests.class})
 public class TestServerCrashProcedure {
-  // Ugly junit parameterization. I just want to pass false and then true but seems like needs
-  // to return sequences of two-element arrays.
-  @Parameters(name = "{index}: setting={0}")
-  public static Collection<Object []> data() {
-    return Arrays.asList(new Object[] [] {{Boolean.FALSE, -1}});
-  }
+  private static final Log LOG = LogFactory.getLog(TestServerCrashProcedure.class);
 
-  private final HBaseTestingUtility util = new HBaseTestingUtility();
+  private HBaseTestingUtility util;
+
+  private void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    conf.set("hbase.balancer.tablesOnMaster", "none");
+    conf.setInt("hbase.client.retries.number", 3);
+  }
 
   @Before
   public void setup() throws Exception {
+    this.util = new HBaseTestingUtility();
+    setupConf(this.util.getConfiguration());
     this.util.startMiniCluster(3);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(
       this.util.getHBaseCluster().getMaster().getMasterProcedureExecutor(), false);
@@ -71,15 +81,25 @@ public class TestServerCrashProcedure {
     MiniHBaseCluster cluster = this.util.getHBaseCluster();
     HMaster master = cluster == null? null: cluster.getMaster();
     if (master != null && master.getMasterProcedureExecutor() != null) {
-      ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(master.getMasterProcedureExecutor(),
-        false);
+      ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(
+        master.getMasterProcedureExecutor(), false);
     }
     this.util.shutdownMiniCluster();
   }
 
-  public TestServerCrashProcedure(final Boolean b, final int ignore) {
-    this.util.getConfiguration().setBoolean("hbase.master.distributed.log.replay", b);
-    this.util.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+
+  @Test(timeout=60000)
+  public void testCrashTargetRs() throws Exception {
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionOnRsWithMeta() throws Exception {
+    testRecoveryAndDoubleExecution(true);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionOnRsWithoutMeta() throws Exception {
+    testRecoveryAndDoubleExecution(false);
   }
 
   /**
@@ -87,43 +107,49 @@ public class TestServerCrashProcedure {
    * needed state.
    * @throws Exception
    */
-  @Test(timeout = 300000)
-  public void testRecoveryAndDoubleExecutionOnline() throws Exception {
-    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
-    this.util.createTable(tableName, HBaseTestingUtility.COLUMNS,
-      HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
-    try (Table t = this.util.getConnection().getTable(tableName)) {
+  private void testRecoveryAndDoubleExecution(final boolean carryingMeta) throws Exception {
+    final TableName tableName = TableName.valueOf(
+      "testRecoveryAndDoubleExecution-carryingMeta-" + carryingMeta);
+    final Table t = this.util.createTable(tableName, HBaseTestingUtility.COLUMNS,
+        HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
+    try {
       // Load the table with a bit of data so some logs to split and some edits in each region.
       this.util.loadTable(t, HBaseTestingUtility.COLUMNS[0]);
-      int count = util.countRows(t);
+      final int count = util.countRows(t);
+      assertTrue("expected some rows", count > 0);
+      final String checksum = util.checksumRows(t);
       // Run the procedure executor outside the master so we can mess with it. Need to disable
       // Master's running of the server crash processing.
-      HMaster master = this.util.getHBaseCluster().getMaster();
+      final HMaster master = this.util.getHBaseCluster().getMaster();
       final ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
       master.setServerCrashProcessingEnabled(false);
-      // Kill a server. Master will notice but do nothing other than add it to list of dead servers.
-      HRegionServer hrs = this.util.getHBaseCluster().getRegionServer(0);
-      boolean carryingMeta = master.getAssignmentManager().isCarryingMeta(hrs.getServerName());
-      this.util.getHBaseCluster().killRegionServer(hrs.getServerName());
-      hrs.join();
-      // Wait until the expiration of the server has arrived at the master. We won't process it
-      // by queuing a ServerCrashProcedure because we have disabled crash processing... but wait
-      // here so ServerManager gets notice and adds expired server to appropriate queues.
-      while (!master.getServerManager().isServerDead(hrs.getServerName())) Threads.sleep(10);
+      // find the first server that match the request and executes the test
+      ServerName rsToKill = null;
+      for (HRegionInfo hri: util.getHBaseAdmin().getTableRegions(tableName)) {
+        final ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(util, hri);
+        if (AssignmentTestingUtil.isServerHoldingMeta(util, serverName) == carryingMeta) {
+          rsToKill = serverName;
+          break;
+        }
+      }
+      // kill the RS
+      AssignmentTestingUtil.killRs(util, rsToKill);
       // Now, reenable processing else we can't get a lock on the ServerCrashProcedure.
       master.setServerCrashProcessingEnabled(true);
       // Do some of the master processing of dead servers so when SCP runs, it has expected 'state'.
-      master.getServerManager().moveFromOnlineToDeadServers(hrs.getServerName());
+      master.getServerManager().moveFromOnlineToDeadServers(rsToKill);
       // Enable test flags and then queue the crash procedure.
       ProcedureTestingUtility.waitNoProcedureRunning(procExec);
       ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-      long procId =
-        procExec.submitProcedure(new ServerCrashProcedure(
-          procExec.getEnvironment(), hrs.getServerName(), true, carryingMeta));
+      long procId = procExec.submitProcedure(new ServerCrashProcedure(
+          procExec.getEnvironment(), rsToKill, true, carryingMeta));
       // Now run through the procedure twice crashing the executor on each step...
       MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
       // Assert all data came back.
       assertEquals(count, util.countRows(t));
+      assertEquals(checksum, util.checksumRows(t));
+    } finally {
+      t.close();
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
deleted file mode 100644
index c3b910e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
+++ /dev/null
@@ -1,420 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.master.procedure;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.CompactionState;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-@Category({MasterTests.class, MediumTests.class})
-public class TestSplitTableRegionProcedure {
-  private static final Log LOG = LogFactory.getLog(TestSplitTableRegionProcedure.class);
-
-  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String ColumnFamilyName1 = "cf1";
-  private static String ColumnFamilyName2 = "cf2";
-
-  private static final int startRowNum = 11;
-  private static final int rowCount = 60;
-
-  @Rule
-  public TestName name = new TestName();
-
-  private static void setupConf(Configuration conf) {
-    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
-    conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 0);
-  }
-
-  @BeforeClass
-  public static void setupCluster() throws Exception {
-    setupConf(UTIL.getConfiguration());
-    UTIL.startMiniCluster(3);
-  }
-
-  @AfterClass
-  public static void cleanupTest() throws Exception {
-    try {
-      UTIL.shutdownMiniCluster();
-    } catch (Exception e) {
-      LOG.warn("failure shutting down cluster", e);
-    }
-  }
-
-  @Before
-  public void setup() throws Exception {
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
-
-    // Turn off balancer so it doesn't cut in and mess up our placements.
-    UTIL.getAdmin().setBalancerRunning(false, true);
-    // Turn off the meta scanner so it don't remove parent on us.
-    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
-      LOG.info("Tear down, remove table=" + htd.getTableName());
-      UTIL.deleteTable(htd.getTableName());
-    }
-  }
-
-  @Test(timeout=60000)
-  public void testSplitTableRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    int splitRowNum = startRowNum + rowCount / 2;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-    // Wait the completion
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    verify(tableName, splitRowNum);
-  }
-
-  @Test(timeout=60000)
-  public void testSplitTableRegionNoStoreFile() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    int splitRowNum = startRowNum + rowCount / 2;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-    // Wait the completion
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    assertTrue(UTIL.getMiniHBaseCluster().getRegions(tableName).size() == 2);
-    assertTrue(UTIL.countRows(tableName) == 0);
-  }
-
-  @Test(timeout=60000)
-  public void testSplitTableRegionUnevenDaughter() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    // Split to two daughters with one of them only has 1 row
-    int splitRowNum = startRowNum + rowCount / 4;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-    // Wait the completion
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    verify(tableName, splitRowNum);
-  }
-
-  @Test(timeout=60000)
-  public void testSplitTableRegionEmptyDaughter() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    // Split to two daughters with one of them only has 1 row
-    int splitRowNum = startRowNum + rowCount;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-    // Wait the completion
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    // Make sure one daughter has 0 rows.
-    List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
-    assertTrue(daughters.size() == 2);
-    assertTrue(UTIL.countRows(tableName) == rowCount);
-    assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
-  }
-
-  @Test(timeout=60000)
-  public void testSplitTableRegionDeletedRowsDaughter() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    // Split to two daughters with one of them only has 1 row
-    int splitRowNum = rowCount;
-    deleteData(tableName, splitRowNum);
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-    // Wait the completion
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    UTIL.getAdmin().majorCompact(tableName);
-    // waiting for the major compaction to complete
-    UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
-      @Override
-      public boolean evaluate() throws IOException {
-        return UTIL.getAdmin().getCompactionState(tableName) == CompactionState.NONE;
-      }
-    });
-
-    // Make sure one daughter has 0 rows.
-    List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
-    assertTrue(daughters.size() == 2);
-    final int currentRowCount = splitRowNum - startRowNum;
-    assertTrue(UTIL.countRows(tableName) == currentRowCount);
-    assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
-  }
-
-  @Test(timeout=60000)
-  public void testInvalidSplitKey() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-
-    // Split region of the table with null split key
-    try {
-      long procId1 = procExec.submitProcedure(
-        new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], null));
-      ProcedureTestingUtility.waitProcedure(procExec, procId1);
-      fail("unexpected procedure start with invalid split-key");
-    } catch (DoNotRetryIOException e) {
-      LOG.debug("Expected Split procedure construction failure: " + e.getMessage());
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testRollbackAndDoubleExecution() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    int splitRowNum = startRowNum + rowCount / 2;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-
-    // Failing before SPLIT_TABLE_REGION_UPDATE_META we should trigger the
-    // rollback
-    // NOTE: the 5 (number before SPLIT_TABLE_REGION_UPDATE_META step) is
-    // hardcoded, so you have to look at this test at least once when you add a new step.
-    int numberOfSteps = 5;
-    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
-      procExec,
-      procId,
-      numberOfSteps);
-  }
-
-  @Test(timeout=60000)
-  public void testRecoveryAndDoubleExecution() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
-
-    HRegionInfo [] regions = MasterProcedureTestingUtility.createTable(
-      procExec, tableName, null, ColumnFamilyName1, ColumnFamilyName2);
-    insertData(tableName);
-    int splitRowNum = startRowNum + rowCount / 2;
-    byte[] splitKey = Bytes.toBytes("" + splitRowNum);
-
-    assertTrue("not able to find a splittable region", regions != null);
-    assertTrue("not able to find a splittable region", regions.length == 1);
-    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
-    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-
-    // Split region of the table
-    long procId = procExec.submitProcedure(
-      new SplitTableRegionProcedure(procExec.getEnvironment(), regions[0], splitKey));
-
-    // Restart the executor and execute the step twice
-    int numberOfSteps = SplitTableRegionState.values().length;
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
-    verify(tableName, splitRowNum);
-  }
-
-  private void insertData(final TableName tableName) throws IOException, InterruptedException {
-    Table t = UTIL.getConnection().getTable(tableName);
-    Put p;
-    for (int i= 0; i < rowCount / 2; i++) {
-      p = new Put(Bytes.toBytes("" + (startRowNum + i)));
-      p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
-      p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
-      t.put(p);
-      p = new Put(Bytes.toBytes("" + (startRowNum + rowCount - i - 1)));
-      p.addColumn(Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes("q1"), Bytes.toBytes(i));
-      p.addColumn(Bytes.toBytes(ColumnFamilyName2), Bytes.toBytes("q2"), Bytes.toBytes(i));
-      t.put(p);
-      if (i % 5 == 0) {
-        UTIL.getAdmin().flush(tableName);
-      }
-    }
-  }
-
-  private void deleteData(
-      final TableName tableName,
-      final int startDeleteRowNum) throws IOException, InterruptedException {
-    Table t = UTIL.getConnection().getTable(tableName);
-    final int numRows = rowCount + startRowNum - startDeleteRowNum;
-    Delete d;
-    for (int i= startDeleteRowNum; i <= numRows + startDeleteRowNum; i++) {
-      d = new Delete(Bytes.toBytes("" + i));
-      t.delete(d);
-      if (i % 5 == 0) {
-        UTIL.getAdmin().flush(tableName);
-      }
-    }
-  }
-
-  private void verify(final TableName tableName, final int splitRowNum) throws IOException {
-    List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
-    assertTrue(daughters.size() == 2);
-    LOG.info("Row Count = " + UTIL.countRows(tableName));
-    assertTrue(UTIL.countRows(tableName) == rowCount);
-    int startRow;
-    int numRows;
-    for (int i = 0; i < daughters.size(); i++) {
-      if (Bytes.compareTo(
-        daughters.get(i).getRegionInfo().getStartKey(), HConstants.EMPTY_BYTE_ARRAY) == 0) {
-        startRow = startRowNum; // first region
-        numRows = splitRowNum - startRowNum;
-      } else {
-        startRow = splitRowNum;
-        numRows = rowCount + startRowNum - splitRowNum;
-      }
-      verifyData(
-        daughters.get(i),
-        startRow,
-        numRows,
-        ColumnFamilyName1.getBytes(),
-        ColumnFamilyName2.getBytes());
-    }
-  }
-
-  private void verifyData(
-      final HRegion newReg,
-      final int startRow,
-      final int numRows,
-      final byte[]... families)
-      throws IOException {
-    for (int i = startRow; i < startRow + numRows; i++) {
-      byte[] row = Bytes.toBytes("" + i);
-      Get get = new Get(row);
-      Result result = newReg.get(get);
-      Cell[] raw = result.rawCells();
-      assertEquals(families.length, result.size());
-      for (int j = 0; j < families.length; j++) {
-        assertTrue(CellUtil.matchingRow(raw[j], row));
-        assertTrue(CellUtil.matchingFamily(raw[j], families[j]));
-      }
-    }
-  }
-
-  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
-    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
index f453a67..f7b4100 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDDLProcedureBase.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.junit.After;
@@ -75,6 +76,10 @@ public abstract class TestTableDDLProcedureBase {
   }
 
   protected ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
-    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    return getMaster().getMasterProcedureExecutor();
+  }
+
+  protected HMaster getMaster() {
+    return UTIL.getHBaseCluster().getMaster();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
index 6d9475f..22583d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableProcedure.java
@@ -18,8 +18,12 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.TableName;
@@ -34,6 +38,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -41,6 +46,8 @@ import static org.junit.Assert.assertTrue;
 @Category({MasterTests.class, MediumTests.class})
 public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
   private static final Log LOG = LogFactory.getLog(TestTruncateTableProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   @Rule
   public TestName name = new TestName();
@@ -171,9 +178,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
       new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
 
     // Restart the executor and execute the step twice
-    // NOTE: the 7 (number of TruncateTableState steps) is hardcoded,
-    //       so you have to look at this test at least once when you add a new step.
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, 7);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
     UTIL.waitUntilAllRegionsAssigned(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index cc79915..7bf518e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -67,7 +67,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;
@@ -729,7 +729,7 @@ public class TestNamespaceAuditor {
     ADMIN.createTable(tableDescOne);
     ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
   }
-  
+
   @Test(expected = QuotaExceededException.class)
   public void testCloneSnapshotQuotaExceed() throws Exception {
     String nsp = prefix + "_testTableQuotaExceedWithCloneSnapshot";