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:05 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14614 [created] 8faab93a9


http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
index 296b38f..8eb2e58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/SimpleMasterProcedureManager.java
@@ -35,7 +35,7 @@ import org.apache.zookeeper.KeeperException;
 
 public class SimpleMasterProcedureManager extends MasterProcedureManager {
 
-  public static final String SIMPLE_SIGNATURE = "simle_test";
+  public static final String SIMPLE_SIGNATURE = "simple_test";
   public static final String SIMPLE_DATA = "simple_test_data";
 
   private static final Log LOG = LogFactory.getLog(SimpleMasterProcedureManager.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
index c7b6c7c..9f3ae8b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
@@ -77,10 +77,10 @@ public class TestCompactSplitThread {
     // block writes if we get to blockingStoreFiles store files
     conf.setInt("hbase.hstore.blockingStoreFiles", blockingStoreFiles);
     // Ensure no extra cleaners on by default (e.g. TimeToLiveHFileCleaner)
-    conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 3);
-    conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 4);
-    conf.setInt(CompactSplitThread.SPLIT_THREADS, 5);
-    conf.setInt(CompactSplitThread.MERGE_THREADS, 6);
+    conf.setInt(CompactSplit.LARGE_COMPACTION_THREADS, 3);
+    conf.setInt(CompactSplit.SMALL_COMPACTION_THREADS, 4);
+    conf.setInt(CompactSplit.SPLIT_THREADS, 5);
+    conf.setInt(CompactSplit.MERGE_THREADS, 6);
   }
 
   @After
@@ -116,10 +116,10 @@ public class TestCompactSplitThread {
       assertEquals(6, regionServer.compactSplitThread.getMergeThreadNum());
 
       // change bigger configurations and do online update
-      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 4);
-      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 5);
-      conf.setInt(CompactSplitThread.SPLIT_THREADS, 6);
-      conf.setInt(CompactSplitThread.MERGE_THREADS, 7);
+      conf.setInt(CompactSplit.LARGE_COMPACTION_THREADS, 4);
+      conf.setInt(CompactSplit.SMALL_COMPACTION_THREADS, 5);
+      conf.setInt(CompactSplit.SPLIT_THREADS, 6);
+      conf.setInt(CompactSplit.MERGE_THREADS, 7);
       try {
         regionServer.compactSplitThread.onConfigurationChange(conf);
       } catch (IllegalArgumentException iae) {
@@ -133,10 +133,10 @@ public class TestCompactSplitThread {
       assertEquals(7, regionServer.compactSplitThread.getMergeThreadNum());
 
       // change smaller configurations and do online update
-      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 2);
-      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 3);
-      conf.setInt(CompactSplitThread.SPLIT_THREADS, 4);
-      conf.setInt(CompactSplitThread.MERGE_THREADS, 5);
+      conf.setInt(CompactSplit.LARGE_COMPACTION_THREADS, 2);
+      conf.setInt(CompactSplit.SMALL_COMPACTION_THREADS, 3);
+      conf.setInt(CompactSplit.SPLIT_THREADS, 4);
+      conf.setInt(CompactSplit.MERGE_THREADS, 5);
       try {
         regionServer.compactSplitThread.onConfigurationChange(conf);
       } catch (IllegalArgumentException iae) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 1bf6ea7..c43ccb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -294,7 +294,7 @@ public class TestCompaction {
     // setup a compact/split thread on a mock server
     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
-    CompactSplitThread thread = new CompactSplitThread(mockServer);
+    CompactSplit thread = new CompactSplit(mockServer);
     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
 
     // setup a region/store with some files
@@ -318,7 +318,7 @@ public class TestCompaction {
     // setup a compact/split thread on a mock server
     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
-    CompactSplitThread thread = new CompactSplitThread(mockServer);
+    CompactSplit thread = new CompactSplit(mockServer);
     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
 
     // setup a region/store with some files
@@ -357,7 +357,7 @@ public class TestCompaction {
 
   /**
    * HBASE-7947: Regression test to ensure adding to the correct list in the
-   * {@link CompactSplitThread}
+   * {@link CompactSplit}
    * @throws Exception on failure
    */
   @Test
@@ -365,7 +365,7 @@ public class TestCompaction {
     // setup a compact/split thread on a mock server
     HRegionServer mockServer = Mockito.mock(HRegionServer.class);
     Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
-    CompactSplitThread thread = new CompactSplitThread(mockServer);
+    CompactSplit thread = new CompactSplit(mockServer);
     Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
 
     // setup a region/store with some files
@@ -548,7 +548,7 @@ public class TestCompaction {
     when(mockServer.isStopped()).thenReturn(false);
     when(mockServer.getConfiguration()).thenReturn(conf);
     when(mockServer.getChoreService()).thenReturn(new ChoreService("test"));
-    CompactSplitThread cst = new CompactSplitThread(mockServer);
+    CompactSplit cst = new CompactSplit(mockServer);
     when(mockServer.getCompactSplitThread()).thenReturn(cst);
     //prevent large compaction thread pool stealing job from small compaction queue.
     cst.shutdownLongCompactions();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
index 88bbffb..0aa39f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
@@ -107,8 +107,8 @@ public class TestHRegionFileSystem {
       // alter through setting HStore#BLOCK_STORAGE_POLICY_KEY in HColumnDescriptor
       hcdA.setValue(HStore.BLOCK_STORAGE_POLICY_KEY, "ONE_SSD");
       admin.modifyColumnFamily(TABLE_NAME, hcdA);
-      while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
-          .isRegionsInTransition()) {
+      while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().
+          getRegionStates().hasRegionsInTransition()) {
         Thread.sleep(200);
         LOG.debug("Waiting on table to finish schema altering");
       }
@@ -117,7 +117,7 @@ public class TestHRegionFileSystem {
       hcdB.setStoragePolicy("ALL_SSD");
       admin.modifyColumnFamily(TABLE_NAME, hcdB);
       while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
-          .isRegionsInTransition()) {
+          .hasRegionsInTransition()) {
         Thread.sleep(200);
         LOG.debug("Waiting on table to finish schema altering");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
index c04f2d4..a7d969a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMergeTransactionOnCluster.java
@@ -56,12 +56,12 @@ 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.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
@@ -168,7 +168,7 @@ public class TestRegionMergeTransactionOnCluster {
     assertTrue(regionStates.isRegionInState(hri, State.MERGED));
 
     // We should not be able to unassign it either
-    am.unassign(hri, null);
+    am.unassign(hri);
     assertFalse("Merged region can't be unassigned",
       regionStates.isRegionInTransition(hri));
     assertTrue(regionStates.isRegionInState(hri, State.MERGED));
@@ -320,12 +320,12 @@ public class TestRegionMergeTransactionOnCluster {
     try {
       // Create table and load data.
       Table table = createTableAndLoadData(MASTER, tableName);
-      RegionStates regionStates = MASTER.getAssignmentManager().getRegionStates();
-      List<HRegionInfo> regions = regionStates.getRegionsOfTable(tableName);
+      AssignmentManager am = MASTER.getAssignmentManager();
+      List<HRegionInfo> regions = am.getRegionStates().getRegionsOfTable(tableName);
       // Fake offline one region
       HRegionInfo a = regions.get(0);
       HRegionInfo b = regions.get(1);
-      regionStates.regionOffline(a);
+      am.offlineRegion(a);
       try {
         // Merge offline region. Region a is offline here
         admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false)
@@ -546,7 +546,7 @@ public class TestRegionMergeTransactionOnCluster {
       if (enabled.get() && req.getTransition(0).getTransitionCode()
           == TransitionCode.READY_TO_MERGE && !resp.hasErrorMessage()) {
         RegionStates regionStates = myMaster.getAssignmentManager().getRegionStates();
-        for (RegionState regionState: regionStates.getRegionsInTransition()) {
+        for (RegionState regionState: regionStates.getRegionsStateInTransition()) {
           // Find the merging_new region and remove it
           if (regionState.isMergingNew()) {
             regionStates.deleteRegion(regionState.getRegion());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 81846df..d9310eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -39,7 +39,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -66,23 +65,24 @@ 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.client.TestReplicasClient.SlowMeCopro;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -103,6 +103,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;
 
 /**
  * The below tests are testing split region against a running cluster
@@ -110,8 +111,9 @@ import org.junit.rules.TestName;
 @Category({RegionServerTests.class, LargeTests.class})
 @SuppressWarnings("deprecation")
 public class TestSplitTransactionOnCluster {
-  private static final Log LOG =
-    LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  private static final Log LOG = LogFactory.getLog(TestSplitTransactionOnCluster.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
   private Admin admin = null;
   private MiniHBaseCluster cluster = null;
   private static final int NB_SERVERS = 3;
@@ -150,8 +152,7 @@ public class TestSplitTransactionOnCluster {
       throws IOException, InterruptedException {
     assertEquals(1, regions.size());
     HRegionInfo hri = regions.get(0).getRegionInfo();
-    cluster.getMaster().getAssignmentManager()
-      .waitOnRegionToClearRegionsInTransition(hri, 600000);
+    cluster.getMaster().getAssignmentManager().waitForAssignment(hri, 600000);
     return hri;
   }
 
@@ -160,21 +161,9 @@ public class TestSplitTransactionOnCluster {
       final Region region,
       final byte[] midKey) throws IOException {
     long procId = cluster.getMaster().splitRegion(region.getRegionInfo(), midKey, 0, 0);
-    // wait
-    if (procId != -1) {
-      // wait for the split to complete or get interrupted.  If the split completes successfully,
-      // the procedure will return true; if the split fails, the procedure would throw exception.
-      //
-      while (!rsServer.isProcedureFinished(procId)) {
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          throw new IOException("Split region interrupted.");
-        }
-      }
-    } else {
-      throw new IOException ("Request split region failed.");
-    }
+    // wait for the split to complete or get interrupted.  If the split completes successfully,
+    // the procedure will return true; if the split fails, the procedure would throw exception.
+    ProcedureTestingUtility.waitProcedure(cluster.getMaster().getMasterProcedureExecutor(), procId);
   }
 
   @Test(timeout = 60000)
@@ -215,7 +204,7 @@ public class TestSplitTransactionOnCluster {
       observer.latch.await();
 
       LOG.info("Waiting for region to come out of RIT");
-      cluster.getMaster().getAssignmentManager().waitOnRegionToClearRegionsInTransition(hri, 60000);
+      cluster.getMaster().getAssignmentManager().waitForAssignment(hri, 60000);
     } finally {
       admin.setBalancerRunning(true, false);
       master.setCatalogJanitorEnabled(true);
@@ -285,7 +274,7 @@ public class TestSplitTransactionOnCluster {
   }
 
   @Test (timeout = 300000)
-  public void testExistingZnodeBlocksSplitAndWeRollback() throws IOException, InterruptedException {
+  public void testSplitRollbackOnRegionClosing() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     // Create table then get the single region for our new table.
@@ -321,8 +310,9 @@ public class TestSplitTransactionOnCluster {
         assertEquals(regionCount, ProtobufUtil.getOnlineRegions(
           server.getRSRpcServices()).size());
       }
+      /* TODO!!!
       regionStates.regionOnline(hri, server.getServerName());
-
+*/
       // Now try splitting and it should work.
       split(hri, server, regionCount);
       // Get daughters
@@ -419,7 +409,7 @@ public class TestSplitTransactionOnCluster {
     }
   }
 
-  @Test(timeout = 180000)
+  @Test(timeout = 60000)
   public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
     TableName userTableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor htd = new HTableDescriptor(userTableName);
@@ -444,6 +434,7 @@ public class TestSplitTransactionOnCluster {
       List<HRegionInfo> regionsOfTable =
           cluster.getMaster().getAssignmentManager().getRegionStates()
           .getRegionsOfTable(userTableName);
+      assertEquals(1, regionsOfTable.size());
       HRegionInfo hRegionInfo = regionsOfTable.get(0);
       Put p = new Put("row6".getBytes());
       p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
@@ -461,10 +452,13 @@ public class TestSplitTransactionOnCluster {
           .getRegionsOfTable(userTableName);
 
       while (regionsOfTable.size() != 2) {
-        Thread.sleep(2000);
+        Thread.sleep(1000);
         regionsOfTable = cluster.getMaster()
             .getAssignmentManager().getRegionStates()
             .getRegionsOfTable(userTableName);
+        LOG.debug("waiting 2 regions to be available, got " + regionsOfTable.size() +
+          ": " + regionsOfTable);
+
       }
       Assert.assertEquals(2, regionsOfTable.size());
 
@@ -526,8 +520,8 @@ public class TestSplitTransactionOnCluster {
       RegionStates regionStates = master.getAssignmentManager().getRegionStates();
       assertTrue("Split parent should be in SPLIT state",
         regionStates.isRegionInState(hri, State.SPLIT));
-      ServerName regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
-      assertTrue(regionServerOfRegion == null);
+      //ServerName regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
+      //assertEquals(null, regionServerOfRegion);
     } finally {
       this.admin.setBalancerRunning(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
@@ -671,7 +665,7 @@ public class TestSplitTransactionOnCluster {
       // Postcondition: split the table with no store files into two regions, but still have not
       // store files
       List<HRegion> daughters = cluster.getRegions(tableName);
-      assertTrue(daughters.size() == 2);
+      assertEquals(2, daughters.size());
 
       // check dirs
       HBaseFsck.debugLsr(conf, new Path("/"));
@@ -685,10 +679,13 @@ public class TestSplitTransactionOnCluster {
       RegionStates regionStates = am.getRegionStates();
       long start = EnvironmentEdgeManager.currentTime();
       while (!regionStates.isRegionInState(hri, State.SPLIT)) {
+        LOG.debug("waiting for SPLIT state on: " + hri);
         assertFalse("Timed out in waiting split parent to be in state SPLIT",
           EnvironmentEdgeManager.currentTime() - start > 60000);
         Thread.sleep(500);
       }
+      assertTrue(regionStates.isRegionInState(daughters.get(0).getRegionInfo(), State.OPEN));
+      assertTrue(regionStates.isRegionInState(daughters.get(1).getRegionInfo(), State.OPEN));
 
       // We should not be able to assign it again
       am.assign(hri, true);
@@ -697,7 +694,7 @@ public class TestSplitTransactionOnCluster {
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
 
       // We should not be able to unassign it either
-      am.unassign(hri, null);
+      am.unassign(hri);
       assertFalse("Split region can't be unassigned",
         regionStates.isRegionInTransition(hri));
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
@@ -939,11 +936,14 @@ public class TestSplitTransactionOnCluster {
       if (enabled.get() && req.getTransition(0).getTransitionCode().equals(
           TransitionCode.READY_TO_SPLIT) && !resp.hasErrorMessage()) {
         RegionStates regionStates = myMaster.getAssignmentManager().getRegionStates();
-        for (RegionState regionState: regionStates.getRegionsInTransition()) {
+        for (RegionStates.RegionStateNode regionState:
+          regionStates.getRegionsInTransition()) {
+          /* TODO!!!!
           // Find the merging_new region and remove it
           if (regionState.isSplittingNew()) {
             regionStates.deleteRegion(regionState.getRegion());
           }
+          */
         }
       }
       return resp;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
index fabf6d2..d384bb5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncLogRolling.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Table;
@@ -31,11 +32,15 @@ import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 
 @Category({ VerySlowRegionServerTests.class, LargeTests.class })
 public class TestAsyncLogRolling extends AbstractTestLogRolling {
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -62,4 +67,8 @@ public class TestAsyncLogRolling extends AbstractTestLogRolling {
     doPut(table, 2);
     assertEquals(numRolledLogFiles + 1, AsyncFSWALProvider.getNumRolledLogFiles(wal));
   }
+
+  public void testLogRolling() throws Exception {
+    super.testLogRolling();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 5bc4c9b..9ccfeef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -58,13 +59,17 @@ import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 
 @Category({ VerySlowRegionServerTests.class, LargeTests.class })
 public class TestLogRolling extends AbstractTestLogRolling {
 
   private static final Log LOG = LogFactory.getLog(TestLogRolling.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java
index 5b8b404..d31d8cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestSecureAsyncWALReplay.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -25,10 +26,14 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider.AsyncWriter;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 
 @Category({ RegionServerTests.class, MediumTests.class })
 public class TestSecureAsyncWALReplay extends TestAsyncWALReplay {
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index e2aa580..d93c782 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -22,16 +22,21 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 
 @Category({ RegionServerTests.class, MediumTests.class })
 public class TestWALReplay extends AbstractTestWALReplay {
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index 02bd49b..f6e328e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -61,8 +61,9 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.mob.MobFileName;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
@@ -91,6 +92,7 @@ public class BaseTestHBaseFsck {
   protected final static String FAM_STR = "fam";
   protected final static byte[] FAM = Bytes.toBytes(FAM_STR);
   protected final static int REGION_ONLINE_TIMEOUT = 800;
+  protected static AssignmentManager assignmentManager;
   protected static RegionStates regionStates;
   protected static ExecutorService tableExecutorService;
   protected static ScheduledThreadPoolExecutor hbfsckExecutorService;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
index b6a185b..ca8bc91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 1d09dfa..5ecfe32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -45,11 +45,10 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
@@ -1595,72 +1594,6 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
     }
   }
 
-  @Test (timeout=180000)
-  public void testCleanUpDaughtersNotInMetaAfterFailedSplit() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
-      createTable(TEST_UTIL, desc, null);
-
-      tbl = connection.getTable(desc.getTableName());
-      for (int i = 0; i < 5; i++) {
-        Put p1 = new Put(("r" + i).getBytes());
-        p1.addColumn(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
-        tbl.put(p1);
-      }
-      admin.flush(desc.getTableName());
-      List<HRegion> regions = cluster.getRegions(desc.getTableName());
-      int serverWith = cluster.getServerWith(regions.get(0).getRegionInfo().getRegionName());
-      HRegionServer regionServer = cluster.getRegionServer(serverWith);
-      byte[] parentRegionName = regions.get(0).getRegionInfo().getRegionName();
-      cluster.getServerWith(parentRegionName);
-      // Create daughters without adding to META table
-      MasterProcedureEnv env = cluster.getMaster().getMasterProcedureExecutor().getEnvironment();
-      SplitTableRegionProcedure splitR = new SplitTableRegionProcedure(
-        env, regions.get(0).getRegionInfo(), Bytes.toBytes("r3"));
-      splitR.prepareSplitRegion(env);
-      splitR.setRegionStateToSplitting(env);
-      splitR.closeParentRegionForSplit(env);
-      splitR.createDaughterRegions(env);
-
-      AssignmentManager am = cluster.getMaster().getAssignmentManager();
-      for (RegionState state : am.getRegionStates().getRegionsInTransition()) {
-        am.regionOffline(state.getRegion());
-      }
-
-      Map<HRegionInfo, ServerName> regionsMap = new HashMap<>();
-      regionsMap.put(regions.get(0).getRegionInfo(), regionServer.getServerName());
-      am.assign(regionsMap);
-      am.waitForAssignment(regions.get(0).getRegionInfo());
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new HBaseFsck.ErrorReporter.ERROR_CODE[] {
-        HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-        HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
-      // holes are separate from overlap groups
-      assertEquals(0, hbck.getOverlapGroups(tableName).size());
-
-      // fix hole
-      assertErrors(
-        doFsck(conf, false, true, false, false, false, false, false, false, false, false, false,
-          false, null),
-        new HBaseFsck.ErrorReporter.ERROR_CODE[] {
-          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-          HBaseFsck.ErrorReporter.ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
-
-      // check that hole fixed
-      assertNoErrors(doFsck(conf, false));
-      assertEquals(5, countRows());
-    } finally {
-      if (tbl != null) {
-        tbl.close();
-        tbl = null;
-      }
-      cleanupTable(tableName);
-    }
-  }
-
   /**
    * This creates fixes a bad table with a hole in meta.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
index 403bf5e..ed39143 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplicas.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.HRegionLocation;
 
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
index 6f90bb2..a058d75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckTwoRS.java
@@ -19,7 +19,13 @@
 
 package org.apache.hadoop.hbase.util;
 
-import com.google.common.collect.Multimap;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.assertNoErrors;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.SynchronousQueue;
@@ -35,8 +41,6 @@ import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -51,8 +55,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
-import static org.junit.Assert.*;
+import com.google.common.collect.Multimap;
 
 @Category({MiscTests.class, LargeTests.class})
 public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
@@ -78,8 +81,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
 
     hbfsckExecutorService = new ScheduledThreadPoolExecutor(POOL_SIZE);
 
-    AssignmentManager assignmentManager =
-        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
+    assignmentManager = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     regionStates = assignmentManager.getRegionStates();
 
     connection = (ClusterConnection) TEST_UTIL.getConnection();
@@ -108,7 +110,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
   public void testFixAssignmentsWhenMETAinTransition() throws Exception {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     admin.closeRegion(cluster.getServerHoldingMeta(), HRegionInfo.FIRST_META_REGIONINFO);
-    regionStates.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
+    assignmentManager.offlineRegion(HRegionInfo.FIRST_META_REGIONINFO);
     new MetaTableLocator().deleteMetaLocation(cluster.getMaster().getZooKeeper());
     assertFalse(regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
     HBaseFsck hbck = doFsck(conf, true);
@@ -393,7 +395,6 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
 
       // Mess it up by creating an overlap
       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-      HMaster master = cluster.getMaster();
       HRegionInfo hriOverlap1 =
           createRegion(tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("AB"));
       TEST_UTIL.assignRegion(hriOverlap1);
@@ -439,7 +440,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
       try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) {
         Put put = new Put(regionName);
         put.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
-            Bytes.toBytes(serverName.getHostAndPort()));
+            Bytes.toBytes(serverName.getAddress().toString()));
         meta.put(put);
       }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/log4j.properties b/hbase-server/src/test/resources/log4j.properties
index c322699..b8a9a2f 100644
--- a/hbase-server/src/test/resources/log4j.properties
+++ b/hbase-server/src/test/resources/log4j.properties
@@ -57,6 +57,7 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L):
 log4j.logger.org.apache.hadoop=WARN
 log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.apache.hadoop.hbase=DEBUG
+log4j.logger.org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure=TRACE
 
 #These settings are workarounds against spurious logs from the minicluster.
 #See HBASE-4709


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
new file mode 100644
index 0000000..1e58b9c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
@@ -0,0 +1,736 @@
+/**
+ *
+ * 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.regionserver;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.conf.ConfigurationManager;
+import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.StealJobQueue;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Compact region on request and then run split if appropriate
+ */
+@InterfaceAudience.Private
+public class CompactSplit implements CompactionRequestor, PropagatingConfigurationObserver {
+  private static final Log LOG = LogFactory.getLog(CompactSplit.class);
+
+  // Configuration key for the large compaction threads.
+  public final static String LARGE_COMPACTION_THREADS =
+      "hbase.regionserver.thread.compaction.large";
+  public final static int LARGE_COMPACTION_THREADS_DEFAULT = 1;
+
+  // Configuration key for the small compaction threads.
+  public final static String SMALL_COMPACTION_THREADS =
+      "hbase.regionserver.thread.compaction.small";
+  public final static int SMALL_COMPACTION_THREADS_DEFAULT = 1;
+
+  // Configuration key for split threads
+  public final static String SPLIT_THREADS = "hbase.regionserver.thread.split";
+  public final static int SPLIT_THREADS_DEFAULT = 1;
+
+  // Configuration keys for merge threads
+  public final static String MERGE_THREADS = "hbase.regionserver.thread.merge";
+  public final static int MERGE_THREADS_DEFAULT = 1;
+
+  public static final String REGION_SERVER_REGION_SPLIT_LIMIT =
+      "hbase.regionserver.regionSplitLimit";
+  public static final int DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT= 1000;
+
+  private final HRegionServer server;
+  private final Configuration conf;
+
+  private final ThreadPoolExecutor longCompactions;
+  private final ThreadPoolExecutor shortCompactions;
+  private final ThreadPoolExecutor splits;
+  private final ThreadPoolExecutor mergePool;
+
+  private volatile ThroughputController compactionThroughputController;
+
+  /**
+   * Splitting should not take place if the total number of regions exceed this.
+   * This is not a hard limit to the number of regions but it is a guideline to
+   * stop splitting after number of online regions is greater than this.
+   */
+  private int regionSplitLimit;
+
+  /** @param server */
+  CompactSplit(HRegionServer server) {
+    super();
+    this.server = server;
+    this.conf = server.getConfiguration();
+    this.regionSplitLimit = conf.getInt(REGION_SERVER_REGION_SPLIT_LIMIT,
+        DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT);
+
+    int largeThreads = Math.max(1, conf.getInt(
+        LARGE_COMPACTION_THREADS, LARGE_COMPACTION_THREADS_DEFAULT));
+    int smallThreads = conf.getInt(
+        SMALL_COMPACTION_THREADS, SMALL_COMPACTION_THREADS_DEFAULT);
+
+    int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
+
+    // if we have throttle threads, make sure the user also specified size
+    Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
+
+    final String n = Thread.currentThread().getName();
+
+    StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<>();
+    this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
+        60, TimeUnit.SECONDS, stealJobQueue,
+        new ThreadFactory() {
+          @Override
+          public Thread newThread(Runnable r) {
+            String name = n + "-longCompactions-" + System.currentTimeMillis();
+            return new Thread(r, name);
+          }
+      });
+    this.longCompactions.setRejectedExecutionHandler(new Rejection());
+    this.longCompactions.prestartAllCoreThreads();
+    this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
+        60, TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
+        new ThreadFactory() {
+          @Override
+          public Thread newThread(Runnable r) {
+            String name = n + "-shortCompactions-" + System.currentTimeMillis();
+            return new Thread(r, name);
+          }
+      });
+    this.shortCompactions
+        .setRejectedExecutionHandler(new Rejection());
+    this.splits = (ThreadPoolExecutor)
+        Executors.newFixedThreadPool(splitThreads,
+            new ThreadFactory() {
+          @Override
+          public Thread newThread(Runnable r) {
+            String name = n + "-splits-" + System.currentTimeMillis();
+            return new Thread(r, name);
+          }
+      });
+    int mergeThreads = conf.getInt(MERGE_THREADS, MERGE_THREADS_DEFAULT);
+    this.mergePool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
+        mergeThreads, new ThreadFactory() {
+          @Override
+          public Thread newThread(Runnable r) {
+            String name = n + "-merges-" + System.currentTimeMillis();
+            return new Thread(r, name);
+          }
+        });
+
+    // compaction throughput controller
+    this.compactionThroughputController =
+        CompactionThroughputControllerFactory.create(server, conf);
+  }
+
+  @Override
+  public String toString() {
+    return "compaction_queue=("
+        + longCompactions.getQueue().size() + ":"
+        + shortCompactions.getQueue().size() + ")"
+        + ", split_queue=" + splits.getQueue().size()
+        + ", merge_queue=" + mergePool.getQueue().size();
+  }
+
+  public String dumpQueue() {
+    StringBuffer queueLists = new StringBuffer();
+    queueLists.append("Compaction/Split Queue dump:\n");
+    queueLists.append("  LargeCompation Queue:\n");
+    BlockingQueue<Runnable> lq = longCompactions.getQueue();
+    Iterator<Runnable> it = lq.iterator();
+    while (it.hasNext()) {
+      queueLists.append("    " + it.next().toString());
+      queueLists.append("\n");
+    }
+
+    if (shortCompactions != null) {
+      queueLists.append("\n");
+      queueLists.append("  SmallCompation Queue:\n");
+      lq = shortCompactions.getQueue();
+      it = lq.iterator();
+      while (it.hasNext()) {
+        queueLists.append("    " + it.next().toString());
+        queueLists.append("\n");
+      }
+    }
+
+    queueLists.append("\n");
+    queueLists.append("  Split Queue:\n");
+    lq = splits.getQueue();
+    it = lq.iterator();
+    while (it.hasNext()) {
+      queueLists.append("    " + it.next().toString());
+      queueLists.append("\n");
+    }
+
+    queueLists.append("\n");
+    queueLists.append("  Region Merge Queue:\n");
+    lq = mergePool.getQueue();
+    it = lq.iterator();
+    while (it.hasNext()) {
+      queueLists.append("    " + it.next().toString());
+      queueLists.append("\n");
+    }
+
+    return queueLists.toString();
+  }
+
+  public synchronized void requestRegionsMerge(final Region a,
+      final Region b, final boolean forcible, long masterSystemTime, User user) {
+    try {
+      mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible, masterSystemTime,user));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Region merge requested for " + a + "," + b + ", forcible="
+            + forcible + ".  " + this);
+      }
+    } catch (RejectedExecutionException ree) {
+      LOG.warn("Could not execute merge for " + a + "," + b + ", forcible="
+          + forcible, ree);
+    }
+  }
+
+  public synchronized boolean requestSplit(final Region r) {
+    // don't split regions that are blocking
+    if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) {
+      byte[] midKey = ((HRegion)r).checkSplit();
+      if (midKey != null) {
+        requestSplit(r, midKey);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public synchronized void requestSplit(final Region r, byte[] midKey) {
+    requestSplit(r, midKey, null);
+  }
+
+  /*
+   * The User parameter allows the split thread to assume the correct user identity
+   */
+  public synchronized void requestSplit(final Region r, byte[] midKey, User user) {
+    if (midKey == null) {
+      LOG.debug("Region " + r.getRegionInfo().getRegionNameAsString() +
+        " not splittable because midkey=null");
+      if (((HRegion)r).shouldForceSplit()) {
+        ((HRegion)r).clearSplit();
+      }
+      return;
+    }
+    try {
+      this.splits.execute(new SplitRequest(r, midKey, this.server, user));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Splitting " + r + ", " + this);
+      }
+    } catch (RejectedExecutionException ree) {
+      LOG.info("Could not execute split for " + r, ree);
+    }
+  }
+
+  @Override
+  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why)
+      throws IOException {
+    return requestCompaction(r, why, null);
+  }
+
+  @Override
+  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
+      List<Pair<CompactionRequest, Store>> requests) throws IOException {
+    return requestCompaction(r, why, Store.NO_PRIORITY, requests, null);
+  }
+
+  @Override
+  public synchronized CompactionRequest requestCompaction(final Region r, final Store s,
+      final String why, CompactionRequest request) throws IOException {
+    return requestCompaction(r, s, why, Store.NO_PRIORITY, request, null);
+  }
+
+  @Override
+  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
+      int p, List<Pair<CompactionRequest, Store>> requests, User user) throws IOException {
+    return requestCompactionInternal(r, why, p, requests, true, user);
+  }
+
+  private List<CompactionRequest> requestCompactionInternal(final Region r, final String why,
+      int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow, User user)
+          throws IOException {
+    // not a special compaction request, so make our own list
+    List<CompactionRequest> ret = null;
+    if (requests == null) {
+      ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
+      for (Store s : r.getStores()) {
+        CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow, user);
+        if (selectNow) ret.add(cr);
+      }
+    } else {
+      Preconditions.checkArgument(selectNow); // only system requests have selectNow == false
+      ret = new ArrayList<CompactionRequest>(requests.size());
+      for (Pair<CompactionRequest, Store> pair : requests) {
+        ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst(), user));
+      }
+    }
+    return ret;
+  }
+
+  public CompactionRequest requestCompaction(final Region r, final Store s,
+      final String why, int priority, CompactionRequest request, User user) throws IOException {
+    return requestCompactionInternal(r, s, why, priority, request, true, user);
+  }
+
+  public synchronized void requestSystemCompaction(
+      final Region r, final String why) throws IOException {
+    requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false, null);
+  }
+
+  public void requestSystemCompaction(
+      final Region r, final Store s, final String why) throws IOException {
+    requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false, null);
+  }
+
+  /**
+   * @param r region store belongs to
+   * @param s Store to request compaction on
+   * @param why Why compaction requested -- used in debug messages
+   * @param priority override the default priority (NO_PRIORITY == decide)
+   * @param request custom compaction request. Can be <tt>null</tt> in which case a simple
+   *          compaction will be used.
+   */
+  private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s,
+      final String why, int priority, CompactionRequest request, boolean selectNow, User user)
+          throws IOException {
+    if (this.server.isStopped()
+        || (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
+      return null;
+    }
+
+    CompactionContext compaction = null;
+    if (selectNow) {
+      compaction = selectCompaction(r, s, priority, request, user);
+      if (compaction == null) return null; // message logged inside
+    }
+
+    // We assume that most compactions are small. So, put system compactions into small
+    // pool; we will do selection there, and move to large pool if necessary.
+    ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
+      ? longCompactions : shortCompactions;
+    pool.execute(new CompactionRunner(s, r, compaction, pool, user));
+    if (LOG.isDebugEnabled()) {
+      String type = (pool == shortCompactions) ? "Small " : "Large ";
+      LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
+          + (why != null && !why.isEmpty() ? "; Because: " + why : "") + "; " + this);
+    }
+    return selectNow ? compaction.getRequest() : null;
+  }
+
+  private CompactionContext selectCompaction(final Region r, final Store s,
+      int priority, CompactionRequest request, User user) throws IOException {
+    CompactionContext compaction = s.requestCompaction(priority, request, user);
+    if (compaction == null) {
+      if(LOG.isDebugEnabled() && r.getRegionInfo() != null) {
+        LOG.debug("Not compacting " + r.getRegionInfo().getRegionNameAsString() +
+            " because compaction request was cancelled");
+      }
+      return null;
+    }
+    assert compaction.hasSelection();
+    if (priority != Store.NO_PRIORITY) {
+      compaction.getRequest().setPriority(priority);
+    }
+    return compaction;
+  }
+
+  /**
+   * Only interrupt once it's done with a run through the work loop.
+   */
+  void interruptIfNecessary() {
+    splits.shutdown();
+    mergePool.shutdown();
+    longCompactions.shutdown();
+    shortCompactions.shutdown();
+  }
+
+  private void waitFor(ThreadPoolExecutor t, String name) {
+    boolean done = false;
+    while (!done) {
+      try {
+        done = t.awaitTermination(60, TimeUnit.SECONDS);
+        LOG.info("Waiting for " + name + " to finish...");
+        if (!done) {
+          t.shutdownNow();
+        }
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted waiting for " + name + " to finish...");
+      }
+    }
+  }
+
+  void join() {
+    waitFor(splits, "Split Thread");
+    waitFor(mergePool, "Merge Thread");
+    waitFor(longCompactions, "Large Compaction Thread");
+    waitFor(shortCompactions, "Small Compaction Thread");
+  }
+
+  /**
+   * Returns the current size of the queue containing regions that are
+   * processed.
+   *
+   * @return The current size of the regions queue.
+   */
+  public int getCompactionQueueSize() {
+    return longCompactions.getQueue().size() + shortCompactions.getQueue().size();
+  }
+
+  public int getLargeCompactionQueueSize() {
+    return longCompactions.getQueue().size();
+  }
+
+
+  public int getSmallCompactionQueueSize() {
+    return shortCompactions.getQueue().size();
+  }
+
+  public int getSplitQueueSize() {
+    return splits.getQueue().size();
+  }
+
+  private boolean shouldSplitRegion() {
+    if(server.getNumberOfOnlineRegions() > 0.9*regionSplitLimit) {
+      LOG.warn("Total number of regions is approaching the upper limit " + regionSplitLimit + ". "
+          + "Please consider taking a look at http://hbase.apache.org/book.html#ops.regionmgt");
+    }
+    return (regionSplitLimit > server.getNumberOfOnlineRegions());
+  }
+
+  /**
+   * @return the regionSplitLimit
+   */
+  public int getRegionSplitLimit() {
+    return this.regionSplitLimit;
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
+      justification="Contrived use of compareTo")
+  private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
+    private final Store store;
+    private final HRegion region;
+    private CompactionContext compaction;
+    private int queuedPriority;
+    private ThreadPoolExecutor parent;
+    private User user;
+
+    public CompactionRunner(Store store, Region region,
+        CompactionContext compaction, ThreadPoolExecutor parent, User user) {
+      super();
+      this.store = store;
+      this.region = (HRegion)region;
+      this.compaction = compaction;
+      this.queuedPriority = (this.compaction == null)
+          ? store.getCompactPriority() : compaction.getRequest().getPriority();
+      this.parent = parent;
+      this.user = user;
+    }
+
+    @Override
+    public String toString() {
+      return (this.compaction != null) ? ("Request = " + compaction.getRequest())
+          : ("Store = " + store.toString() + ", pri = " + queuedPriority);
+    }
+
+    private void doCompaction(User user) {
+      // Common case - system compaction without a file selection. Select now.
+      if (this.compaction == null) {
+        int oldPriority = this.queuedPriority;
+        this.queuedPriority = this.store.getCompactPriority();
+        if (this.queuedPriority > oldPriority) {
+          // Store priority decreased while we were in queue (due to some other compaction?),
+          // requeue with new priority to avoid blocking potential higher priorities.
+          this.parent.execute(this);
+          return;
+        }
+        try {
+          this.compaction = selectCompaction(this.region, this.store, queuedPriority, null, user);
+        } catch (IOException ex) {
+          LOG.error("Compaction selection failed " + this, ex);
+          server.checkFileSystem();
+          return;
+        }
+        if (this.compaction == null) return; // nothing to do
+        // Now see if we are in correct pool for the size; if not, go to the correct one.
+        // We might end up waiting for a while, so cancel the selection.
+        assert this.compaction.hasSelection();
+        ThreadPoolExecutor pool = store.throttleCompaction(
+            compaction.getRequest().getSize()) ? longCompactions : shortCompactions;
+
+        // Long compaction pool can process small job
+        // Short compaction pool should not process large job
+        if (this.parent == shortCompactions && pool == longCompactions) {
+          this.store.cancelRequestedCompaction(this.compaction);
+          this.compaction = null;
+          this.parent = pool;
+          this.parent.execute(this);
+          return;
+        }
+      }
+      // Finally we can compact something.
+      assert this.compaction != null;
+
+      this.compaction.getRequest().beforeExecute();
+      try {
+        // Note: please don't put single-compaction logic here;
+        //       put it into region/store/etc. This is CST logic.
+        long start = EnvironmentEdgeManager.currentTime();
+        boolean completed =
+            region.compact(compaction, store, compactionThroughputController, user);
+        long now = EnvironmentEdgeManager.currentTime();
+        LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
+              this + "; duration=" + StringUtils.formatTimeDiff(now, start));
+        if (completed) {
+          // degenerate case: blocked regions require recursive enqueues
+          if (store.getCompactPriority() <= 0) {
+            requestSystemCompaction(region, store, "Recursive enqueue");
+          } else {
+            // see if the compaction has caused us to exceed max region size
+            requestSplit(region);
+          }
+        }
+      } catch (IOException ex) {
+        IOException remoteEx =
+            ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
+        LOG.error("Compaction failed " + this, remoteEx);
+        if (remoteEx != ex) {
+          LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
+        }
+        region.reportCompactionRequestFailure();
+        server.checkFileSystem();
+      } catch (Exception ex) {
+        LOG.error("Compaction failed " + this, ex);
+        region.reportCompactionRequestFailure();
+        server.checkFileSystem();
+      } finally {
+        LOG.debug("CompactSplitThread Status: " + CompactSplit.this);
+      }
+      this.compaction.getRequest().afterExecute();
+    }
+
+    @Override
+    public void run() {
+      Preconditions.checkNotNull(server);
+      if (server.isStopped()
+          || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
+        return;
+      }
+      doCompaction(user);
+    }
+
+    private String formatStackTrace(Exception ex) {
+      StringWriter sw = new StringWriter();
+      PrintWriter pw = new PrintWriter(sw);
+      ex.printStackTrace(pw);
+      pw.flush();
+      return sw.toString();
+    }
+
+    @Override
+    public int compareTo(CompactionRunner o) {
+      // Only compare the underlying request (if any), for queue sorting purposes.
+      int compareVal = queuedPriority - o.queuedPriority; // compare priority
+      if (compareVal != 0) return compareVal;
+      CompactionContext tc = this.compaction, oc = o.compaction;
+      // Sort pre-selected (user?) compactions before system ones with equal priority.
+      return (tc == null) ? ((oc == null) ? 0 : 1)
+          : ((oc == null) ? -1 : tc.getRequest().compareTo(oc.getRequest()));
+    }
+  }
+
+  /**
+   * Cleanup class to use when rejecting a compaction request from the queue.
+   */
+  private static class Rejection implements RejectedExecutionHandler {
+    @Override
+    public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
+      if (runnable instanceof CompactionRunner) {
+        CompactionRunner runner = (CompactionRunner)runnable;
+        LOG.debug("Compaction Rejected: " + runner);
+        runner.store.cancelRequestedCompaction(runner.compaction);
+      }
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void onConfigurationChange(Configuration newConf) {
+    // Check if number of large / small compaction threads has changed, and then
+    // adjust the core pool size of the thread pools, by using the
+    // setCorePoolSize() method. According to the javadocs, it is safe to
+    // change the core pool size on-the-fly. We need to reset the maximum
+    // pool size, as well.
+    int largeThreads = Math.max(1, newConf.getInt(
+            LARGE_COMPACTION_THREADS,
+            LARGE_COMPACTION_THREADS_DEFAULT));
+    if (this.longCompactions.getCorePoolSize() != largeThreads) {
+      LOG.info("Changing the value of " + LARGE_COMPACTION_THREADS +
+              " from " + this.longCompactions.getCorePoolSize() + " to " +
+              largeThreads);
+      if(this.longCompactions.getCorePoolSize() < largeThreads) {
+        this.longCompactions.setMaximumPoolSize(largeThreads);
+        this.longCompactions.setCorePoolSize(largeThreads);
+      } else {
+        this.longCompactions.setCorePoolSize(largeThreads);
+        this.longCompactions.setMaximumPoolSize(largeThreads);
+      }
+    }
+
+    int smallThreads = newConf.getInt(SMALL_COMPACTION_THREADS,
+            SMALL_COMPACTION_THREADS_DEFAULT);
+    if (this.shortCompactions.getCorePoolSize() != smallThreads) {
+      LOG.info("Changing the value of " + SMALL_COMPACTION_THREADS +
+                " from " + this.shortCompactions.getCorePoolSize() + " to " +
+                smallThreads);
+      if(this.shortCompactions.getCorePoolSize() < smallThreads) {
+        this.shortCompactions.setMaximumPoolSize(smallThreads);
+        this.shortCompactions.setCorePoolSize(smallThreads);
+      } else {
+        this.shortCompactions.setCorePoolSize(smallThreads);
+        this.shortCompactions.setMaximumPoolSize(smallThreads);
+      }
+    }
+
+    int splitThreads = newConf.getInt(SPLIT_THREADS,
+            SPLIT_THREADS_DEFAULT);
+    if (this.splits.getCorePoolSize() != splitThreads) {
+      LOG.info("Changing the value of " + SPLIT_THREADS +
+                " from " + this.splits.getCorePoolSize() + " to " +
+                splitThreads);
+      if(this.splits.getCorePoolSize() < splitThreads) {
+        this.splits.setMaximumPoolSize(splitThreads);
+        this.splits.setCorePoolSize(splitThreads);
+      } else {
+        this.splits.setCorePoolSize(splitThreads);
+        this.splits.setMaximumPoolSize(splitThreads);
+      }
+    }
+
+    int mergeThreads = newConf.getInt(MERGE_THREADS,
+            MERGE_THREADS_DEFAULT);
+    if (this.mergePool.getCorePoolSize() != mergeThreads) {
+      LOG.info("Changing the value of " + MERGE_THREADS +
+                " from " + this.mergePool.getCorePoolSize() + " to " +
+                mergeThreads);
+      if(this.mergePool.getCorePoolSize() < mergeThreads) {
+        this.mergePool.setMaximumPoolSize(mergeThreads);
+        this.mergePool.setCorePoolSize(mergeThreads);
+      } else {
+        this.mergePool.setCorePoolSize(mergeThreads);
+        this.mergePool.setMaximumPoolSize(mergeThreads);
+      }
+    }
+
+    ThroughputController old = this.compactionThroughputController;
+    if (old != null) {
+      old.stop("configuration change");
+    }
+    this.compactionThroughputController =
+        CompactionThroughputControllerFactory.create(server, newConf);
+
+    // We change this atomically here instead of reloading the config in order that upstream
+    // would be the only one with the flexibility to reload the config.
+    this.conf.reloadConfiguration();
+  }
+
+  protected int getSmallCompactionThreadNum() {
+    return this.shortCompactions.getCorePoolSize();
+  }
+
+  protected int getLargeCompactionThreadNum() {
+    return this.longCompactions.getCorePoolSize();
+  }
+
+  protected int getSplitThreadNum() {
+    return this.splits.getCorePoolSize();
+  }
+
+  protected int getMergeThreadNum() {
+    return this.mergePool.getCorePoolSize();
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void registerChildren(ConfigurationManager manager) {
+    // No children to register.
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void deregisterChildren(ConfigurationManager manager) {
+    // No children to register
+  }
+
+  @VisibleForTesting
+  public ThroughputController getCompactionThroughputController() {
+    return compactionThroughputController;
+  }
+
+  @VisibleForTesting
+  public long getCompletedMergeTaskCount() {
+    return mergePool.getCompletedTaskCount();
+  }
+
+  @VisibleForTesting
+  /**
+   * Shutdown the long compaction thread pool.
+   * Should only be used in unit test to prevent long compaction thread pool from stealing job
+   * from short compaction queue
+   */
+  void shutdownLongCompactions(){
+    this.longCompactions.shutdown();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
deleted file mode 100644
index eba984a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ /dev/null
@@ -1,722 +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.regionserver;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.RejectedExecutionHandler;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.conf.ConfigurationManager;
-import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
-import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.StealJobQueue;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-/**
- * Compact region on request and then run split if appropriate
- */
-@InterfaceAudience.Private
-public class CompactSplitThread implements CompactionRequestor, PropagatingConfigurationObserver {
-  private static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
-
-  // Configuration key for the large compaction threads.
-  public final static String LARGE_COMPACTION_THREADS =
-      "hbase.regionserver.thread.compaction.large";
-  public final static int LARGE_COMPACTION_THREADS_DEFAULT = 1;
-  
-  // Configuration key for the small compaction threads.
-  public final static String SMALL_COMPACTION_THREADS =
-      "hbase.regionserver.thread.compaction.small";
-  public final static int SMALL_COMPACTION_THREADS_DEFAULT = 1;
-  
-  // Configuration key for split threads
-  public final static String SPLIT_THREADS = "hbase.regionserver.thread.split";
-  public final static int SPLIT_THREADS_DEFAULT = 1;
-  
-  // Configuration keys for merge threads
-  public final static String MERGE_THREADS = "hbase.regionserver.thread.merge";
-  public final static int MERGE_THREADS_DEFAULT = 1;
-
-  public static final String REGION_SERVER_REGION_SPLIT_LIMIT =
-      "hbase.regionserver.regionSplitLimit";
-  public static final int DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT= 1000;
-
-  private final HRegionServer server;
-  private final Configuration conf;
-
-  private final ThreadPoolExecutor longCompactions;
-  private final ThreadPoolExecutor shortCompactions;
-  private final ThreadPoolExecutor splits;
-  private final ThreadPoolExecutor mergePool;
-
-  private volatile ThroughputController compactionThroughputController;
-
-  /**
-   * Splitting should not take place if the total number of regions exceed this.
-   * This is not a hard limit to the number of regions but it is a guideline to
-   * stop splitting after number of online regions is greater than this.
-   */
-  private int regionSplitLimit;
-
-  /** @param server */
-  CompactSplitThread(HRegionServer server) {
-    super();
-    this.server = server;
-    this.conf = server.getConfiguration();
-    this.regionSplitLimit = conf.getInt(REGION_SERVER_REGION_SPLIT_LIMIT,
-        DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT);
-
-    int largeThreads = Math.max(1, conf.getInt(
-        LARGE_COMPACTION_THREADS, LARGE_COMPACTION_THREADS_DEFAULT));
-    int smallThreads = conf.getInt(
-        SMALL_COMPACTION_THREADS, SMALL_COMPACTION_THREADS_DEFAULT);
-
-    int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
-
-    // if we have throttle threads, make sure the user also specified size
-    Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
-
-    final String n = Thread.currentThread().getName();
-
-    StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<>();
-    this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
-        60, TimeUnit.SECONDS, stealJobQueue,
-        new ThreadFactory() {
-          @Override
-          public Thread newThread(Runnable r) {
-            String name = n + "-longCompactions-" + System.currentTimeMillis();
-            return new Thread(r, name);
-          }
-      });
-    this.longCompactions.setRejectedExecutionHandler(new Rejection());
-    this.longCompactions.prestartAllCoreThreads();
-    this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
-        60, TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
-        new ThreadFactory() {
-          @Override
-          public Thread newThread(Runnable r) {
-            String name = n + "-shortCompactions-" + System.currentTimeMillis();
-            return new Thread(r, name);
-          }
-      });
-    this.shortCompactions
-        .setRejectedExecutionHandler(new Rejection());
-    this.splits = (ThreadPoolExecutor)
-        Executors.newFixedThreadPool(splitThreads,
-            new ThreadFactory() {
-          @Override
-          public Thread newThread(Runnable r) {
-            String name = n + "-splits-" + System.currentTimeMillis();
-            return new Thread(r, name);
-          }
-      });
-    int mergeThreads = conf.getInt(MERGE_THREADS, MERGE_THREADS_DEFAULT);
-    this.mergePool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
-        mergeThreads, new ThreadFactory() {
-          @Override
-          public Thread newThread(Runnable r) {
-            String name = n + "-merges-" + System.currentTimeMillis();
-            return new Thread(r, name);
-          }
-        });
-
-    // compaction throughput controller
-    this.compactionThroughputController =
-        CompactionThroughputControllerFactory.create(server, conf);
-  }
-
-  @Override
-  public String toString() {
-    return "compaction_queue=("
-        + longCompactions.getQueue().size() + ":"
-        + shortCompactions.getQueue().size() + ")"
-        + ", split_queue=" + splits.getQueue().size()
-        + ", merge_queue=" + mergePool.getQueue().size();
-  }
-  
-  public String dumpQueue() {
-    StringBuffer queueLists = new StringBuffer();
-    queueLists.append("Compaction/Split Queue dump:\n");
-    queueLists.append("  LargeCompation Queue:\n");
-    BlockingQueue<Runnable> lq = longCompactions.getQueue();
-    Iterator<Runnable> it = lq.iterator();
-    while (it.hasNext()) {
-      queueLists.append("    " + it.next().toString());
-      queueLists.append("\n");
-    }
-
-    if (shortCompactions != null) {
-      queueLists.append("\n");
-      queueLists.append("  SmallCompation Queue:\n");
-      lq = shortCompactions.getQueue();
-      it = lq.iterator();
-      while (it.hasNext()) {
-        queueLists.append("    " + it.next().toString());
-        queueLists.append("\n");
-      }
-    }
-
-    queueLists.append("\n");
-    queueLists.append("  Split Queue:\n");
-    lq = splits.getQueue();
-    it = lq.iterator();
-    while (it.hasNext()) {
-      queueLists.append("    " + it.next().toString());
-      queueLists.append("\n");
-    }
-
-    queueLists.append("\n");
-    queueLists.append("  Region Merge Queue:\n");
-    lq = mergePool.getQueue();
-    it = lq.iterator();
-    while (it.hasNext()) {
-      queueLists.append("    " + it.next().toString());
-      queueLists.append("\n");
-    }
-
-    return queueLists.toString();
-  }
-
-  public synchronized boolean requestSplit(final Region r) {
-    // don't split regions that are blocking
-    if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) {
-      byte[] midKey = ((HRegion)r).checkSplit();
-      if (midKey != null) {
-        requestSplit(r, midKey);
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public synchronized void requestSplit(final Region r, byte[] midKey) {
-    requestSplit(r, midKey, null);
-  }
-
-  /*
-   * The User parameter allows the split thread to assume the correct user identity
-   */
-  public synchronized void requestSplit(final Region r, byte[] midKey, User user) {
-    if (midKey == null) {
-      LOG.debug("Region " + r.getRegionInfo().getRegionNameAsString() +
-        " not splittable because midkey=null");
-      if (((HRegion)r).shouldForceSplit()) {
-        ((HRegion)r).clearSplit();
-      }
-      return;
-    }
-    try {
-      this.splits.execute(new SplitRequest(r, midKey, this.server, user));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Split requested for " + r + ".  " + this);
-      }
-    } catch (RejectedExecutionException ree) {
-      LOG.info("Could not execute split for " + r, ree);
-    }
-  }
-
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why)
-      throws IOException {
-    return requestCompaction(r, why, null);
-  }
-
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
-      List<Pair<CompactionRequest, Store>> requests) throws IOException {
-    return requestCompaction(r, why, Store.NO_PRIORITY, requests, null);
-  }
-
-  @Override
-  public synchronized CompactionRequest requestCompaction(final Region r, final Store s,
-      final String why, CompactionRequest request) throws IOException {
-    return requestCompaction(r, s, why, Store.NO_PRIORITY, request, null);
-  }
-
-  @Override
-  public synchronized List<CompactionRequest> requestCompaction(final Region r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests, User user) throws IOException {
-    return requestCompactionInternal(r, why, p, requests, true, user);
-  }
-
-  private List<CompactionRequest> requestCompactionInternal(final Region r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow, User user)
-          throws IOException {
-    // not a special compaction request, so make our own list
-    List<CompactionRequest> ret = null;
-    if (requests == null) {
-      ret = selectNow ? new ArrayList<>(r.getStores().size()) : null;
-      for (Store s : r.getStores()) {
-        CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow, user);
-        if (selectNow) ret.add(cr);
-      }
-    } else {
-      Preconditions.checkArgument(selectNow); // only system requests have selectNow == false
-      ret = new ArrayList<>(requests.size());
-      for (Pair<CompactionRequest, Store> pair : requests) {
-        ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst(), user));
-      }
-    }
-    return ret;
-  }
-
-  public CompactionRequest requestCompaction(final Region r, final Store s,
-      final String why, int priority, CompactionRequest request, User user) throws IOException {
-    return requestCompactionInternal(r, s, why, priority, request, true, user);
-  }
-
-  public synchronized void requestSystemCompaction(
-      final Region r, final String why) throws IOException {
-    requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false, null);
-  }
-
-  public void requestSystemCompaction(
-      final Region r, final Store s, final String why) throws IOException {
-    requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false, null);
-  }
-
-  /**
-   * @param r region store belongs to
-   * @param s Store to request compaction on
-   * @param why Why compaction requested -- used in debug messages
-   * @param priority override the default priority (NO_PRIORITY == decide)
-   * @param request custom compaction request. Can be <tt>null</tt> in which case a simple
-   *          compaction will be used.
-   */
-  private synchronized CompactionRequest requestCompactionInternal(final Region r, final Store s,
-      final String why, int priority, CompactionRequest request, boolean selectNow, User user)
-          throws IOException {
-    if (this.server.isStopped()
-        || (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
-      return null;
-    }
-
-    CompactionContext compaction = null;
-    if (selectNow) {
-      compaction = selectCompaction(r, s, priority, request, user);
-      if (compaction == null) return null; // message logged inside
-    }
-
-    // We assume that most compactions are small. So, put system compactions into small
-    // pool; we will do selection there, and move to large pool if necessary.
-    ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
-      ? longCompactions : shortCompactions;
-    pool.execute(new CompactionRunner(s, r, compaction, pool, user));
-    if (LOG.isDebugEnabled()) {
-      String type = (pool == shortCompactions) ? "Small " : "Large ";
-      LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
-          + (why != null && !why.isEmpty() ? "; Because: " + why : "") + "; " + this);
-    }
-    return selectNow ? compaction.getRequest() : null;
-  }
-
-  private CompactionContext selectCompaction(final Region r, final Store s,
-      int priority, CompactionRequest request, User user) throws IOException {
-    CompactionContext compaction = s.requestCompaction(priority, request, user);
-    if (compaction == null) {
-      if(LOG.isDebugEnabled() && r.getRegionInfo() != null) {
-        LOG.debug("Not compacting " + r.getRegionInfo().getRegionNameAsString() +
-            " because compaction request was cancelled");
-      }
-      return null;
-    }
-    assert compaction.hasSelection();
-    if (priority != Store.NO_PRIORITY) {
-      compaction.getRequest().setPriority(priority);
-    }
-    return compaction;
-  }
-
-  /**
-   * Only interrupt once it's done with a run through the work loop.
-   */
-  void interruptIfNecessary() {
-    splits.shutdown();
-    mergePool.shutdown();
-    longCompactions.shutdown();
-    shortCompactions.shutdown();
-  }
-
-  private void waitFor(ThreadPoolExecutor t, String name) {
-    boolean done = false;
-    while (!done) {
-      try {
-        done = t.awaitTermination(60, TimeUnit.SECONDS);
-        LOG.info("Waiting for " + name + " to finish...");
-        if (!done) {
-          t.shutdownNow();
-        }
-      } catch (InterruptedException ie) {
-        LOG.warn("Interrupted waiting for " + name + " to finish...");
-      }
-    }
-  }
-
-  void join() {
-    waitFor(splits, "Split Thread");
-    waitFor(mergePool, "Merge Thread");
-    waitFor(longCompactions, "Large Compaction Thread");
-    waitFor(shortCompactions, "Small Compaction Thread");
-  }
-
-  /**
-   * Returns the current size of the queue containing regions that are
-   * processed.
-   *
-   * @return The current size of the regions queue.
-   */
-  public int getCompactionQueueSize() {
-    return longCompactions.getQueue().size() + shortCompactions.getQueue().size();
-  }
-
-  public int getLargeCompactionQueueSize() {
-    return longCompactions.getQueue().size();
-  }
-
-
-  public int getSmallCompactionQueueSize() {
-    return shortCompactions.getQueue().size();
-  }
-
-  public int getSplitQueueSize() {
-    return splits.getQueue().size();
-  }
-
-  private boolean shouldSplitRegion() {
-    if(server.getNumberOfOnlineRegions() > 0.9*regionSplitLimit) {
-      LOG.warn("Total number of regions is approaching the upper limit " + regionSplitLimit + ". "
-          + "Please consider taking a look at http://hbase.apache.org/book.html#ops.regionmgt");
-    }
-    return (regionSplitLimit > server.getNumberOfOnlineRegions());
-  }
-
-  /**
-   * @return the regionSplitLimit
-   */
-  public int getRegionSplitLimit() {
-    return this.regionSplitLimit;
-  }
-
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
-      justification="Contrived use of compareTo")
-  private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
-    private final Store store;
-    private final HRegion region;
-    private CompactionContext compaction;
-    private int queuedPriority;
-    private ThreadPoolExecutor parent;
-    private User user;
-
-    public CompactionRunner(Store store, Region region,
-        CompactionContext compaction, ThreadPoolExecutor parent, User user) {
-      super();
-      this.store = store;
-      this.region = (HRegion)region;
-      this.compaction = compaction;
-      this.queuedPriority = (this.compaction == null)
-          ? store.getCompactPriority() : compaction.getRequest().getPriority();
-      this.parent = parent;
-      this.user = user;
-    }
-
-    @Override
-    public String toString() {
-      return (this.compaction != null) ? ("Request = " + compaction.getRequest())
-          : ("Store = " + store.toString() + ", pri = " + queuedPriority);
-    }
-
-    private void doCompaction(User user) {
-      // Common case - system compaction without a file selection. Select now.
-      if (this.compaction == null) {
-        int oldPriority = this.queuedPriority;
-        this.queuedPriority = this.store.getCompactPriority();
-        if (this.queuedPriority > oldPriority) {
-          // Store priority decreased while we were in queue (due to some other compaction?),
-          // requeue with new priority to avoid blocking potential higher priorities.
-          this.parent.execute(this);
-          return;
-        }
-        try {
-          this.compaction = selectCompaction(this.region, this.store, queuedPriority, null, user);
-        } catch (IOException ex) {
-          LOG.error("Compaction selection failed " + this, ex);
-          server.checkFileSystem();
-          return;
-        }
-        if (this.compaction == null) return; // nothing to do
-        // Now see if we are in correct pool for the size; if not, go to the correct one.
-        // We might end up waiting for a while, so cancel the selection.
-        assert this.compaction.hasSelection();
-        ThreadPoolExecutor pool = store.throttleCompaction(
-            compaction.getRequest().getSize()) ? longCompactions : shortCompactions;
-
-        // Long compaction pool can process small job
-        // Short compaction pool should not process large job
-        if (this.parent == shortCompactions && pool == longCompactions) {
-          this.store.cancelRequestedCompaction(this.compaction);
-          this.compaction = null;
-          this.parent = pool;
-          this.parent.execute(this);
-          return;
-        }
-      }
-      // Finally we can compact something.
-      assert this.compaction != null;
-
-      this.compaction.getRequest().beforeExecute();
-      try {
-        // Note: please don't put single-compaction logic here;
-        //       put it into region/store/etc. This is CST logic.
-        long start = EnvironmentEdgeManager.currentTime();
-        boolean completed =
-            region.compact(compaction, store, compactionThroughputController, user);
-        long now = EnvironmentEdgeManager.currentTime();
-        LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
-              this + "; duration=" + StringUtils.formatTimeDiff(now, start));
-        if (completed) {
-          // degenerate case: blocked regions require recursive enqueues
-          if (store.getCompactPriority() <= 0) {
-            requestSystemCompaction(region, store, "Recursive enqueue");
-          } else {
-            // see if the compaction has caused us to exceed max region size
-            requestSplit(region);
-          }
-        }
-      } catch (IOException ex) {
-        IOException remoteEx =
-            ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;
-        LOG.error("Compaction failed " + this, remoteEx);
-        if (remoteEx != ex) {
-          LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
-        }
-        region.reportCompactionRequestFailure();
-        server.checkFileSystem();
-      } catch (Exception ex) {
-        LOG.error("Compaction failed " + this, ex);
-        region.reportCompactionRequestFailure();
-        server.checkFileSystem();
-      } finally {
-        LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);
-      }
-      this.compaction.getRequest().afterExecute();
-    }
-
-    @Override
-    public void run() {
-      Preconditions.checkNotNull(server);
-      if (server.isStopped()
-          || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
-        return;
-      }
-      doCompaction(user);
-    }
-
-    private String formatStackTrace(Exception ex) {
-      StringWriter sw = new StringWriter();
-      PrintWriter pw = new PrintWriter(sw);
-      ex.printStackTrace(pw);
-      pw.flush();
-      return sw.toString();
-    }
-
-    @Override
-    public int compareTo(CompactionRunner o) {
-      // Only compare the underlying request (if any), for queue sorting purposes.
-      int compareVal = queuedPriority - o.queuedPriority; // compare priority
-      if (compareVal != 0) return compareVal;
-      CompactionContext tc = this.compaction, oc = o.compaction;
-      // Sort pre-selected (user?) compactions before system ones with equal priority.
-      return (tc == null) ? ((oc == null) ? 0 : 1)
-          : ((oc == null) ? -1 : tc.getRequest().compareTo(oc.getRequest()));
-    }
-  }
-
-  /**
-   * Cleanup class to use when rejecting a compaction request from the queue.
-   */
-  private static class Rejection implements RejectedExecutionHandler {
-    @Override
-    public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
-      if (runnable instanceof CompactionRunner) {
-        CompactionRunner runner = (CompactionRunner)runnable;
-        LOG.debug("Compaction Rejected: " + runner);
-        runner.store.cancelRequestedCompaction(runner.compaction);
-      }
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void onConfigurationChange(Configuration newConf) {
-    // Check if number of large / small compaction threads has changed, and then
-    // adjust the core pool size of the thread pools, by using the
-    // setCorePoolSize() method. According to the javadocs, it is safe to
-    // change the core pool size on-the-fly. We need to reset the maximum
-    // pool size, as well.
-    int largeThreads = Math.max(1, newConf.getInt(
-            LARGE_COMPACTION_THREADS,
-            LARGE_COMPACTION_THREADS_DEFAULT));
-    if (this.longCompactions.getCorePoolSize() != largeThreads) {
-      LOG.info("Changing the value of " + LARGE_COMPACTION_THREADS +
-              " from " + this.longCompactions.getCorePoolSize() + " to " +
-              largeThreads);
-      if(this.longCompactions.getCorePoolSize() < largeThreads) {
-        this.longCompactions.setMaximumPoolSize(largeThreads);
-        this.longCompactions.setCorePoolSize(largeThreads);
-      } else {
-        this.longCompactions.setCorePoolSize(largeThreads);
-        this.longCompactions.setMaximumPoolSize(largeThreads);
-      }
-    }
-
-    int smallThreads = newConf.getInt(SMALL_COMPACTION_THREADS,
-            SMALL_COMPACTION_THREADS_DEFAULT);
-    if (this.shortCompactions.getCorePoolSize() != smallThreads) {
-      LOG.info("Changing the value of " + SMALL_COMPACTION_THREADS +
-                " from " + this.shortCompactions.getCorePoolSize() + " to " +
-                smallThreads);
-      if(this.shortCompactions.getCorePoolSize() < smallThreads) {
-        this.shortCompactions.setMaximumPoolSize(smallThreads);
-        this.shortCompactions.setCorePoolSize(smallThreads);
-      } else {
-        this.shortCompactions.setCorePoolSize(smallThreads);
-        this.shortCompactions.setMaximumPoolSize(smallThreads);
-      }
-    }
-
-    int splitThreads = newConf.getInt(SPLIT_THREADS,
-            SPLIT_THREADS_DEFAULT);
-    if (this.splits.getCorePoolSize() != splitThreads) {
-      LOG.info("Changing the value of " + SPLIT_THREADS +
-                " from " + this.splits.getCorePoolSize() + " to " +
-                splitThreads);
-      if(this.splits.getCorePoolSize() < splitThreads) {
-        this.splits.setMaximumPoolSize(splitThreads);
-        this.splits.setCorePoolSize(splitThreads);
-      } else {
-        this.splits.setCorePoolSize(splitThreads);
-        this.splits.setMaximumPoolSize(splitThreads);
-      }
-    }
-
-    int mergeThreads = newConf.getInt(MERGE_THREADS,
-            MERGE_THREADS_DEFAULT);
-    if (this.mergePool.getCorePoolSize() != mergeThreads) {
-      LOG.info("Changing the value of " + MERGE_THREADS +
-                " from " + this.mergePool.getCorePoolSize() + " to " +
-                mergeThreads);
-      if(this.mergePool.getCorePoolSize() < mergeThreads) {
-        this.mergePool.setMaximumPoolSize(mergeThreads);
-        this.mergePool.setCorePoolSize(mergeThreads);
-      } else {
-        this.mergePool.setCorePoolSize(mergeThreads);
-        this.mergePool.setMaximumPoolSize(mergeThreads);
-      }
-    }
-
-    ThroughputController old = this.compactionThroughputController;
-    if (old != null) {
-      old.stop("configuration change");
-    }
-    this.compactionThroughputController =
-        CompactionThroughputControllerFactory.create(server, newConf);
-
-    // We change this atomically here instead of reloading the config in order that upstream
-    // would be the only one with the flexibility to reload the config.
-    this.conf.reloadConfiguration();
-  }
-
-  protected int getSmallCompactionThreadNum() {
-    return this.shortCompactions.getCorePoolSize();
-  }
-
-  protected int getLargeCompactionThreadNum() {
-    return this.longCompactions.getCorePoolSize();
-  }
-
-  protected int getSplitThreadNum() {
-    return this.splits.getCorePoolSize();
-  }
-
-  protected int getMergeThreadNum() {
-    return this.mergePool.getCorePoolSize();
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void registerChildren(ConfigurationManager manager) {
-    // No children to register.
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void deregisterChildren(ConfigurationManager manager) {
-    // No children to register
-  }
-
-  @VisibleForTesting
-  public ThroughputController getCompactionThroughputController() {
-    return compactionThroughputController;
-  }
-
-  @VisibleForTesting
-  public long getCompletedMergeTaskCount() {
-    return mergePool.getCompletedTaskCount();
-  }
-
-  @VisibleForTesting
-  /**
-   * Shutdown the long compaction thread pool.
-   * Should only be used in unit test to prevent long compaction thread pool from stealing job
-   * from short compaction queue
-   */
-  void shutdownLongCompactions(){
-    this.longCompactions.shutdown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index b3b5113..2d31f3c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
-import org.apache.hadoop.hbase.client.NonceGenerator;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
@@ -148,8 +147,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServe
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -161,8 +158,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -170,7 +165,6 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.JSONBean;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
@@ -199,13 +193,13 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
-import sun.misc.Signal;
-import sun.misc.SignalHandler;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
 /**
  * HRegionServer makes a set of HRegions available to clients. It checks in with
  * the HMaster. There are many HRegionServers in a single HBase deployment.
@@ -274,7 +268,7 @@ public class HRegionServer extends HasThread implements
   protected ReplicationSinkService replicationSinkHandler;
 
   // Compactions
-  public CompactSplitThread compactSplitThread;
+  public CompactSplit compactSplitThread;
 
   /**
    * Map of regions currently being served by this region server. Key is the
@@ -902,7 +896,7 @@ public class HRegionServer extends HasThread implements
     this.cacheFlusher = new MemStoreFlusher(conf, this);
 
     // Compaction thread
-    this.compactSplitThread = new CompactSplitThread(this);
+    this.compactSplitThread = new CompactSplit(this);
 
     // Background thread to check for compactions; needed if region has not gotten updates
     // in a while. It will take care of not checking too frequently on store-by-store basis.
@@ -1684,7 +1678,7 @@ public class HRegionServer extends HasThread implements
     final static int RANGE_OF_DELAY = 5 * 60 * 1000; // 5 min in milliseconds
     final static int MIN_DELAY_TIME = 0; // millisec
     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
-      super(server.getServerName() + "-MemstoreFlusherChore", server, cacheFlushInterval);
+      super("MemstoreFlusherChore", server, cacheFlushInterval);
       this.server = server;
     }
 
@@ -2110,6 +2104,7 @@ public class HRegionServer extends HasThread implements
             + " to " + code + ": " + response.getErrorMessage());
           return false;
         }
+        LOG.info("TRANSITION REPORTED " + request);
         return true;
       } catch (ServiceException se) {
         IOException ioe = ProtobufUtil.getRemoteException(se);
@@ -2119,84 +2114,10 @@ public class HRegionServer extends HasThread implements
         }
       }
     }
+    LOG.info("TRANSITION NOT REPORTED " + request);
     return false;
   }
 
-  @Override
-  public long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow) {
-    NonceGenerator ng = clusterConnection.getNonceGenerator();
-    final long nonceGroup = ng.getNonceGroup();
-    final long nonce = ng.newNonce();
-    long procId = -1;
-    SplitTableRegionRequest request =
-        RequestConverter.buildSplitTableRegionRequest(regionInfo, splitRow, nonceGroup, nonce);
-
-    while (keepLooping()) {
-      RegionServerStatusService.BlockingInterface rss = rssStub;
-      try {
-        if (rss == null) {
-          createRegionServerStatusStub();
-          continue;
-        }
-        SplitTableRegionResponse response = rss.splitRegion(null, request);
-
-        //TODO: should we limit the retry number before quitting?
-        if (response == null || (procId = response.getProcId()) == -1) {
-          LOG.warn("Failed to split " + regionInfo + " retrying...");
-          continue;
-        }
-
-        break;
-      } catch (ServiceException se) {
-        // TODO: retry or just fail
-        IOException ioe = ProtobufUtil.getRemoteException(se);
-        LOG.info("Failed to split region, will retry", ioe);
-        if (rssStub == rss) {
-          rssStub = null;
-        }
-      }
-    }
-    return procId;
-  }
-
-  @Override
-  public boolean isProcedureFinished(final long procId) throws IOException {
-    GetProcedureResultRequest request =
-        GetProcedureResultRequest.newBuilder().setProcId(procId).build();
-
-    while (keepLooping()) {
-      RegionServerStatusService.BlockingInterface rss = rssStub;
-      try {
-        if (rss == null) {
-          createRegionServerStatusStub();
-          continue;
-        }
-        // TODO: find a way to get proc result
-        GetProcedureResultResponse response = rss.getProcedureResult(null, request);
-
-        if (response == null) {
-          LOG.warn("Failed to get procedure (id=" + procId + ") status.");
-          return false;
-        } else if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
-          return false;
-        } else if (response.hasException()) {
-          // Procedure failed.
-          throw ForeignExceptionUtil.toIOException(response.getException());
-        }
-        // Procedure completes successfully
-        break;
-      } catch (ServiceException se) {
-        // TODO: retry or just fail
-        IOException ioe = ProtobufUtil.getRemoteException(se);
-        LOG.warn("Failed to get split region procedure result.  Retrying", ioe);
-        if (rssStub == rss) {
-          rssStub = null;
-        }
-      }
-    }
-    return true;
-  }
-
   /**
    * Trigger a flush in the primary region replica if this region is a secondary replica. Does not
    * block this thread. See RegionReplicaFlushHandler for details.
@@ -3419,9 +3340,9 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return the underlying {@link CompactSplitThread} for the servers
+   * @return the underlying {@link CompactSplit} for the servers
    */
-  public CompactSplitThread getCompactSplitThread() {
+  public CompactSplit getCompactSplitThread() {
     return this.compactSplitThread;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 298f538..705442a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -118,12 +119,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -136,6 +137,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
@@ -1399,36 +1402,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
-  @Override
-  @QosPriority(priority=HConstants.ADMIN_QOS)
-  public CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
-      final RpcController controller,
-      final CloseRegionForSplitOrMergeRequest request) throws ServiceException {
-    try {
-      checkOpen();
-
-      List<String> encodedRegionNameList = new ArrayList<>();
-      for(int i = 0; i < request.getRegionCount(); i++) {
-        final String encodedRegionName = ProtobufUtil.getRegionEncodedName(request.getRegion(i));
-
-        // Can be null if we're calling close on a region that's not online
-        final Region targetRegion = regionServer.getFromOnlineRegions(encodedRegionName);
-        if ((targetRegion != null) && (targetRegion.getCoprocessorHost() != null)) {
-          targetRegion.getCoprocessorHost().preClose(false);
-          encodedRegionNameList.add(encodedRegionName);
-        }
-      }
-      requestCount.increment();
-      LOG.info("Close and offline " + encodedRegionNameList + " regions.");
-      boolean closed = regionServer.closeAndOfflineRegionForSplitOrMerge(encodedRegionNameList);
-      CloseRegionForSplitOrMergeResponse.Builder builder =
-          CloseRegionForSplitOrMergeResponse.newBuilder().setClosed(closed);
-      return builder.build();
-    } catch (IOException ie) {
-      throw new ServiceException(ie);
-    }
-  }
-
   /**
    * Compact a region on the region server.
    *
@@ -1742,8 +1715,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           // The region is already online. This should not happen any more.
           String error = "Received OPEN for the region:"
             + region.getRegionNameAsString() + ", which is already online";
-          regionServer.abort(error);
-          throw new IOException(error);
+          LOG.warn(error);
+          //regionServer.abort(error);
+          //throw new IOException(error);
+          builder.addOpeningState(RegionOpeningState.OPENED);
+          continue;
         }
         LOG.info("Open " + region.getRegionNameAsString());
 
@@ -3230,4 +3206,60 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return UpdateConfigurationResponse.getDefaultInstance();
   }
 
-}
+  @Override
+  public ExecuteProceduresResponse executeProcedures(RpcController controller,
+      ExecuteProceduresRequest request) throws ServiceException {
+    ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
+    if (request.getOpenRegionCount() > 0) {
+      for (OpenRegionRequest req: request.getOpenRegionList()) {
+        builder.addOpenRegion(openRegion(controller, req));
+      }
+    }
+    if (request.getCloseRegionCount() > 0) {
+      for (CloseRegionRequest req: request.getCloseRegionList()) {
+        builder.addCloseRegion(closeRegion(controller, req));
+      }
+    }
+    return builder.build();
+  }
+
+  /**
+   * Merge regions on the region server.
+   *
+   * @param controller the RPC controller
+   * @param request the request
+   * @return merge regions response
+   * @throws ServiceException
+   */
+  @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
+  public MergeRegionsResponse mergeRegions(final RpcController controller,
+      final MergeRegionsRequest request) throws ServiceException {
+    try {
+      checkOpen();
+      requestCount.increment();
+      Region regionA = getRegion(request.getRegionA());
+      Region regionB = getRegion(request.getRegionB());
+      boolean forcible = request.getForcible();
+      long masterSystemTime = request.hasMasterSystemTime() ? request.getMasterSystemTime() : -1;
+      regionA.startRegionOperation(Operation.MERGE_REGION);
+      regionB.startRegionOperation(Operation.MERGE_REGION);
+      if (regionA.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
+          regionB.getRegionInfo().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        throw new ServiceException(new MergeRegionException("Can't merge non-default replicas"));
+      }
+      LOG.info("Receiving merging request for  " + regionA + ", " + regionB
+          + ",forcible=" + forcible);
+      regionA.flush(true);
+      regionB.flush(true);
+      regionServer.compactSplitThread.requestRegionsMerge(regionA, regionB, forcible,
+          masterSystemTime, RpcServer.getRequestUser());
+      return MergeRegionsResponse.newBuilder().build();
+    } catch (DroppedSnapshotException ex) {
+      regionServer.abort("Replay of WAL required. Forcing server shutdown", ex);
+      throw new ServiceException(ex);
+    } catch (IOException ie) {
+      throw new ServiceException(ie);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
new file mode 100644
index 0000000..e0980d2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
@@ -0,0 +1,109 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.regionserver;
+
+import java.io.IOException;
+import java.security.PrivilegedAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Handles processing region merges. Put in a queue, owned by HRegionServer.
+ */
+@InterfaceAudience.Private
+class RegionMergeRequest implements Runnable {
+  private static final Log LOG = LogFactory.getLog(RegionMergeRequest.class);
+  private final HRegionInfo region_a;
+  private final HRegionInfo region_b;
+  private final HRegionServer server;
+  private final boolean forcible;
+  private final User user;
+
+  RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible,
+      long masterSystemTime, User user) {
+    Preconditions.checkNotNull(hrs);
+    this.region_a = a.getRegionInfo();
+    this.region_b = b.getRegionInfo();
+    this.server = hrs;
+    this.forcible = forcible;
+    this.user = user;
+  }
+
+  @Override
+  public String toString() {
+    return "MergeRequest,regions:" + region_a + ", " + region_b + ", forcible="
+        + forcible;
+  }
+
+  private void doMerge() {
+    boolean success = false;
+    //server.metricsRegionServer.incrMergeRequest();
+
+    if (user != null && user.getUGI() != null) {
+      user.getUGI().doAs (new PrivilegedAction<Void>() {
+        @Override
+        public Void run() {
+          requestRegionMerge();
+          return null;
+        }
+      });
+    } else {
+      requestRegionMerge();
+    }
+  }
+
+  private void requestRegionMerge() {
+    final TableName table = region_a.getTable();
+    if (!table.equals(region_b.getTable())) {
+      LOG.error("Can't merge regions from two different tables: " + region_a + ", " + region_b);
+      return;
+    }
+
+    // TODO: fake merged region for compat with the report protocol
+    final HRegionInfo merged = new HRegionInfo(table);
+
+    // Send the split request to the master. the master will do the validation on the split-key.
+    // The parent region will be unassigned and the two new regions will be assigned.
+    // hri_a and hri_b objects may not reflect the regions that will be created, those objectes
+    // are created just to pass the information to the reportRegionStateTransition().
+    if (!server.reportRegionStateTransition(TransitionCode.READY_TO_MERGE, merged, region_a, region_b)) {
+      LOG.error("Unable to ask master to merge: " + region_a + ", " + region_b);
+    }
+  }
+
+  @Override
+  public void run() {
+    if (this.server.isStopping() || this.server.isStopped()) {
+      LOG.debug("Skipping merge because server is stopping="
+          + this.server.isStopping() + " or stopped=" + this.server.isStopped());
+      return;
+    }
+
+    doMerge();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 3382263..623eab2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -177,16 +177,6 @@ public interface RegionServerServices extends OnlineRegions, FavoredNodesForRegi
   boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris);
 
   /**
-   * Notify master that a region wants to be splitted.
-   */
-  long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow);
-
-  /**
-   * Check with master whether a procedure is completed (either succeed or fail)
-   */
-  boolean isProcedureFinished(final long procId) throws IOException;
-
-  /**
    * Returns a reference to the region server's RPC server
    */
   RpcServerInterface getRpcServer();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
index eb9811d..5407cfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
@@ -23,8 +23,11 @@ import java.security.PrivilegedAction;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.ipc.RemoteException;
@@ -37,14 +40,14 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 class SplitRequest implements Runnable {
   private static final Log LOG = LogFactory.getLog(SplitRequest.class);
-  private final HRegion parent;
+  private final HRegionInfo parent;
   private final byte[] midKey;
   private final HRegionServer server;
   private final User user;
 
   SplitRequest(Region region, byte[] midKey, HRegionServer hrs, User user) {
     Preconditions.checkNotNull(hrs);
-    this.parent = (HRegion)region;
+    this.parent = region.getRegionInfo();
     this.midKey = midKey;
     this.server = hrs;
     this.user = user;
@@ -58,65 +61,29 @@ class SplitRequest implements Runnable {
   private void doSplitting() {
     boolean success = false;
     server.metricsRegionServer.incrSplitRequest();
-    long startTime = EnvironmentEdgeManager.currentTime();
-
-    try {
-      long procId;
-      if (user != null && user.getUGI() != null) {
-        procId = user.getUGI().doAs (new PrivilegedAction<Long>() {
-          @Override
-          public Long run() {
-            try {
-              return server.requestRegionSplit(parent.getRegionInfo(), midKey);
-            } catch (Exception e) {
-              LOG.error("Failed to complete region split ", e);
-            }
-            return (long)-1;
-          }
-        });
-      } else {
-        procId = server.requestRegionSplit(parent.getRegionInfo(), midKey);
-      }
-
-      if (procId != -1) {
-        // wait for the split to complete or get interrupted.  If the split completes successfully,
-        // the procedure will return true; if the split fails, the procedure would throw exception.
-        //
-        try {
-          while (!(success = server.isProcedureFinished(procId))) {
-            try {
-              Thread.sleep(1000);
-            } catch (InterruptedException e) {
-              LOG.warn("Split region " + parent + " is still in progress.  Not waiting...");
-              break;
-            }
-          }
-        } catch (IOException e) {
-          LOG.error("Split region " + parent + " failed.", e);
+    if (user != null && user.getUGI() != null) {
+      user.getUGI().doAs (new PrivilegedAction<Void>() {
+        @Override
+        public Void run() {
+          requestRegionSplit();
+          return null;
         }
-      } else {
-        LOG.error("Fail to split region " + parent);
-      }
-    } finally {
-      if (this.parent.getCoprocessorHost() != null) {
-        try {
-          this.parent.getCoprocessorHost().postCompleteSplit();
-        } catch (IOException io) {
-          LOG.error("Split failed " + this,
-            io instanceof RemoteException ? ((RemoteException) io).unwrapRemoteException() : io);
-        }
-      }
-
-      // Update regionserver metrics with the split transaction total running time
-      server.metricsRegionServer.updateSplitTime(EnvironmentEdgeManager.currentTime() - startTime);
-
-      if (parent.shouldForceSplit()) {
-        parent.clearSplit();
-      }
+      });
+    } else {
+      requestRegionSplit();
+    }
+  }
 
-      if (success) {
-        server.metricsRegionServer.incrSplitSuccess();
-      }
+  private void requestRegionSplit() {
+    final TableName table = parent.getTable();
+    final HRegionInfo hri_a = new HRegionInfo(table, parent.getStartKey(), midKey);
+    final HRegionInfo hri_b = new HRegionInfo(table, midKey, parent.getEndKey());
+    // Send the split request to the master. the master will do the validation on the split-key.
+    // The parent region will be unassigned and the two new regions will be assigned.
+    // hri_a and hri_b objects may not reflect the regions that will be created, those objectes
+    // are created just to pass the information to the reportRegionStateTransition().
+    if (!server.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT, parent, hri_a, hri_b)) {
+      LOG.error("Unable to ask master to split " + parent.getRegionNameAsString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 4eab62b..91cd258 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -559,7 +559,7 @@ public class HBaseFsck extends Configured implements Closeable {
     errors.print("Number of requests: " + status.getRequestsCount());
     errors.print("Number of regions: " + status.getRegionsCount());
 
-    Set<RegionState> rits = status.getRegionsInTransition();
+    List<RegionState> rits = status.getRegionsInTransition();
     errors.print("Number of regions in transition: " + rits.size());
     if (details) {
       for (RegionState state: rits) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index d7749c2..8ea7012 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 
 /**
  * Utility methods for interacting with the regions.
@@ -223,7 +223,7 @@ public abstract class ModifyRegionUtils {
   static ThreadPoolExecutor getRegionOpenAndInitThreadPool(final Configuration conf,
       final String threadNamePrefix, int regionNumber) {
     int maxThreads = Math.min(regionNumber, conf.getInt(
-        "hbase.hregion.open.and.init.threads.max", 10));
+        "hbase.hregion.open.and.init.threads.max", 16));
     ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
     .getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
         new ThreadFactory() {
@@ -236,24 +236,4 @@ public abstract class ModifyRegionUtils {
         });
     return regionOpenAndInitThreadPool;
   }
-
-  /**
-   * Triggers a bulk assignment of the specified regions
-   *
-   * @param assignmentManager the Assignment Manger
-   * @param regionInfos the list of regions to assign
-   * @throws IOException if an error occurred during the assignment
-   */
-  public static void assignRegions(final AssignmentManager assignmentManager,
-      final List<HRegionInfo> regionInfos) throws IOException {
-    try {
-      assignmentManager.getRegionStates().createRegionStates(regionInfos);
-      assignmentManager.assign(regionInfos);
-    } catch (InterruptedException e) {
-      LOG.error("Caught " + e + " during round-robin assignment");
-      InterruptedIOException ie = new InterruptedIOException(e.getMessage());
-      ie.initCause(e);
-      throw ie;
-    }
-  }
 }


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

Posted by st...@apache.org.
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";


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

Posted by st...@apache.org.
HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)

    Includes four patches from Matteos' repository and then fix up to get it all to
    pass, fix findbugs, etc.. I apply the two patches in one go because applying
    each independently puts hbase in a non-working state.

     1. HBASE-14616 Procedure v2 - Replace the old AM with the new AM
     This comes from Matteo's repo here:
     https://github.com/matteobertozzi/hbase/commit/689227fcbfe8e6588433dbcdabf4526e3d478b2e

     Patch replaces old AM with the new under subpackage master.assignment.
     Mostly just updating classes to use new AM -- import changes -- rather
     than the old. It also removes old AM and supporting classes.
     See below for more detail.

     2. HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)
     Adds running of remote procedure. Adds batching of remote calls.
     Adds support for assign/unassign in procedures. Adds version info
     reporting in rpc. Adds start of an AMv2.

     3. and 4. are fixes around merge and split.

     This work mostly comes from:
     https://github.com/matteobertozzi/hbase/commit/3622cba4e331d2fc7bfc1932abb4c9cbf5802efa

     Reporting of remote RS version is from here:
     https://github.com/matteobertozzi/hbase/commit/ddb4df3964e8298c88c0210e83493aa91ac0942d.patch

     And remote dispatch of procedures is from:
     https://github.com/matteobertozzi/hbase/commit/186b9e7c4dae61a79509a6c3aad7f80ec61345e5

     The split merge patches from here are also melded in:
    https://github.com/matteobertozzi/hbase/commit/9a3a95a2c2974842a4849d1ad867e70764e7f707
    and https://github.com/matteobertozzi/hbase/commit/d6289307a02a777299f65238238a2a8af3253067

     Adds testing util for new AM and new sets of tests.

     Details:

     M hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
     Takes list of regionstates on construction rather than a Set on construction.
     NOTE!!!!! This is a change in a public class.

     M hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
     Purge old overlapping states: PENDING_OPEN, PENDING_CLOSE, etc.

     A hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
     Dispatch remote procedures every 150ms or 32 items -- which ever
     happens first (configurable). Runs a timeout thread.
     Carries notion of a remote procedure and of a buffer full of these.
     "hbase.procedure.remote.dispatcher.threadpool.size" with default = 128
     "hbase.procedure.remote.dispatcher.delay.msec" with default = 150ms
     "hbase.procedure.remote.dispatcher.max.queue.size" with default = 32

     M hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
     Add execute procedures call ExecuteProcedures.

     M hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
     Add assign and unassign state support for procedures.

     M hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
     Adds getting RS version out of RPC
     Examples: (1.3.4 is 0x0103004, 2.1.0 is 0x0201000)

     M hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
     Remove periodic metrics chore. This is done over in new AM now.
     Replace AM with the new.

     M hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
     Have AMv2 handle assigning meta.

     M hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
     Extract version number of the server making rpc.

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
     Add new assign procedure. Runs assign via Procedure Dispatch.
     There can only be one RegionTransitionProcedure per region running at the time,
     since each procedure takes a lock on the region.

     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java

     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
     D hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
     A procedure-based AM (AMv2).

     TODO
      - handle region migration
      - handle meta assignment first
      - handle sys table assignment first (e.g. acl, namespace)
      - handle table priorities
       "hbase.assignment.bootstrap.thread.pool.size"; default size is 16.
       "hbase.assignment.dispatch.wait.msec"; default wait is 150
       "hbase.assignment.dispatch.wait.queue.max.size"; wait max default is 100
       "hbase.assignment.rit.chore.interval.msec"; default is 5 * 1000;
       "hbase.assignment.maximum.attempts"; default is 10;

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
     Procedure that runs subprocedure to unassign and then assign to new location

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
     Manage store of region state (in hbase:meta by default).

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
     In-memory state of all regions. Used by AMv2.

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
     Base RIT procedure for Assign and Unassign.

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
     Unassign procedure.

     A hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
     Run region assignement in a manner that pays attention to target server version.
     Adds "hbase.regionserver.rpc.startup.waittime"; defaults 60 seconds.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8faab93a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8faab93a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8faab93a

Branch: refs/heads/HBASE-14614
Commit: 8faab93a9c5a25ea141d6d750aa31776ef078f66
Parents: f1c1f25
Author: Michael Stack <st...@apache.org>
Authored: Wed Mar 22 09:31:14 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Mar 23 08:42:10 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ClusterStatus.java  |     8 +-
 .../apache/hadoop/hbase/MetaTableAccessor.java  |     2 +-
 .../hbase/client/ConnectionImplementation.java  |    12 +
 .../client/ShortCircuitMasterConnection.java    |    13 +-
 .../hbase/ipc/ServerTooBusyException.java       |     7 +-
 .../apache/hadoop/hbase/master/RegionState.java |    24 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |     2 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   125 +-
 .../hbase/shaded/protobuf/RequestConverter.java |    16 +-
 .../shaded/protobuf/ResponseConverter.java      |    13 -
 .../org/apache/hadoop/hbase/ChoreService.java   |     2 +-
 .../org/apache/hadoop/hbase/HConstants.java     |     2 +-
 .../java/org/apache/hadoop/hbase/TableName.java |     7 +-
 .../hadoop/hbase/zookeeper/TestZKConfig.java    |     1 +
 .../master/MetricsAssignmentManagerSource.java  |    23 +-
 .../MetricsAssignmentManagerSourceImpl.java     |    38 +-
 .../procedure2/AbstractProcedureScheduler.java  |     4 +-
 .../hadoop/hbase/procedure2/LockAndQueue.java   |    29 +-
 .../hadoop/hbase/procedure2/Procedure.java      |     6 +-
 .../hadoop/hbase/procedure2/ProcedureEvent.java |     6 +-
 .../hbase/procedure2/ProcedureExecutor.java     |    18 +-
 .../hbase/procedure2/ProcedureScheduler.java    |     4 +-
 .../procedure2/RemoteProcedureDispatcher.java   |   367 +
 .../hbase/procedure2/StateMachineProcedure.java |     3 +
 .../hbase/procedure2/util/DelayedUtil.java      |    58 +-
 .../procedure2/ProcedureTestingUtility.java     |    12 +-
 .../hbase/procedure2/util/TestDelayedUtil.java  |     2 +-
 .../shaded/protobuf/generated/AdminProtos.java  | 17559 ++++++++++-------
 .../generated/MasterProcedureProtos.java        |  7446 +++++--
 .../shaded/protobuf/generated/MasterProtos.java |  6937 +++++--
 .../generated/RegionServerStatusProtos.java     |  1633 +-
 .../src/main/protobuf/Admin.proto               |    47 +-
 .../src/main/protobuf/Master.proto              |    36 +
 .../src/main/protobuf/MasterProcedure.proto     |    95 +-
 .../src/main/protobuf/RegionServerStatus.proto  |    27 -
 .../hbase/rsgroup/RSGroupAdminServer.java       |    12 +-
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java |     2 +-
 .../balancer/TestRSGroupBasedLoadBalancer.java  |     2 +-
 .../master/AssignmentManagerStatusTmpl.jamon    |    51 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |     2 +-
 .../hadoop/hbase/client/VersionInfoUtil.java    |    81 +-
 .../hbase/ipc/BalancedQueueRpcExecutor.java     |     3 -
 .../ipc/FastPathBalancedQueueRpcExecutor.java   |     2 +-
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java    |    10 +-
 .../hadoop/hbase/ipc/SimpleRpcServer.java       |     4 +-
 .../hadoop/hbase/master/AssignCallable.java     |    49 -
 .../hadoop/hbase/master/AssignmentManager.java  |  3053 ---
 .../hadoop/hbase/master/BulkAssigner.java       |   122 -
 .../apache/hadoop/hbase/master/BulkReOpen.java  |   136 -
 .../hadoop/hbase/master/CatalogJanitor.java     |     6 +-
 .../hbase/master/GeneralBulkAssigner.java       |   213 -
 .../org/apache/hadoop/hbase/master/HMaster.java |   165 +-
 .../hadoop/hbase/master/LoadBalancer.java       |     4 +-
 .../hbase/master/MasterCoprocessorHost.java     |    22 +
 .../hadoop/hbase/master/MasterDumpServlet.java  |     8 +-
 .../hbase/master/MasterMetaBootstrap.java       |    43 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   259 +-
 .../hadoop/hbase/master/MasterServices.java     |    21 +
 .../hadoop/hbase/master/MasterWalManager.java   |    11 +-
 .../hbase/master/MetricsAssignmentManager.java  |    39 +-
 .../hadoop/hbase/master/RegionStateStore.java   |   268 -
 .../hadoop/hbase/master/RegionStates.java       |  1170 --
 .../hadoop/hbase/master/ServerManager.java      |    75 +-
 .../hbase/master/TableNamespaceManager.java     |     5 +-
 .../hadoop/hbase/master/UnAssignCallable.java   |    47 -
 .../master/assignment/AssignProcedure.java      |   270 +
 .../master/assignment/AssignmentManager.java    |  1660 ++
 .../assignment/MergeTableRegionsProcedure.java  |   717 +
 .../master/assignment/MoveRegionProcedure.java  |   147 +
 .../master/assignment/RegionStateStore.java     |   327 +
 .../hbase/master/assignment/RegionStates.java   |   864 +
 .../assignment/RegionTransitionProcedure.java   |   315 +
 .../assignment/SplitTableRegionProcedure.java   |   731 +
 .../master/assignment/UnassignProcedure.java    |   216 +
 .../hbase/master/balancer/BaseLoadBalancer.java |     2 +-
 .../master/balancer/RegionLocationFinder.java   |    14 +-
 .../master/balancer/SimpleLoadBalancer.java     |     9 +-
 .../master/balancer/StochasticLoadBalancer.java |    17 +-
 .../hbase/master/locking/LockProcedure.java     |    36 +-
 .../AbstractStateMachineTableProcedure.java     |     2 +-
 .../procedure/AddColumnFamilyProcedure.java     |    31 +-
 .../procedure/CloneSnapshotProcedure.java       |     4 +-
 .../procedure/CreateNamespaceProcedure.java     |     1 -
 .../master/procedure/CreateTableProcedure.java  |    41 +-
 .../procedure/DeleteColumnFamilyProcedure.java  |    31 +-
 .../master/procedure/DeleteTableProcedure.java  |    10 +-
 .../master/procedure/DisableTableProcedure.java |   154 +-
 .../DispatchMergingRegionsProcedure.java        |   584 +
 .../master/procedure/EnableTableProcedure.java  |   172 +-
 .../procedure/MasterDDLOperationHelper.java     |    93 +-
 .../procedure/MasterProcedureConstants.java     |     2 +-
 .../master/procedure/MasterProcedureEnv.java    |    28 +-
 .../procedure/MasterProcedureScheduler.java     |    95 +-
 .../procedure/MergeTableRegionsProcedure.java   |   909 -
 .../procedure/ModifyColumnFamilyProcedure.java  |    30 +-
 .../master/procedure/ModifyTableProcedure.java  |    30 +-
 .../master/procedure/ProcedureSyncWait.java     |   144 +-
 .../master/procedure/RSProcedureDispatcher.java |   542 +
 .../procedure/RestoreSnapshotProcedure.java     |    27 +-
 .../master/procedure/ServerCrashProcedure.java  |   492 +-
 .../procedure/SplitTableRegionProcedure.java    |   785 -
 .../procedure/TruncateTableProcedure.java       |     6 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |    18 +-
 .../hadoop/hbase/regionserver/CompactSplit.java |   736 +
 .../hbase/regionserver/CompactSplitThread.java  |   722 -
 .../hbase/regionserver/HRegionServer.java       |    99 +-
 .../hbase/regionserver/RSRpcServices.java       |   102 +-
 .../hbase/regionserver/RegionMergeRequest.java  |   109 +
 .../regionserver/RegionServerServices.java      |    10 -
 .../hadoop/hbase/regionserver/SplitRequest.java |    85 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |     2 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |    24 +-
 .../apache/hadoop/hbase/wal/WALSplitter.java    |     5 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |     7 +-
 .../hadoop/hbase/MockRegionServerServices.java  |    10 -
 .../hadoop/hbase/TestRegionRebalancing.java     |    13 +-
 .../hbase/TestStochasticBalancerJmxMetrics.java |     2 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |    20 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |     4 +-
 .../hadoop/hbase/client/TestEnableTable.java    |    34 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java |   119 +-
 .../hbase/client/TestMetaWithReplicas.java      |     4 +-
 .../client/TestScannersFromClientSide.java      |    10 +-
 .../hbase/client/TestServerBusyException.java   |   234 +
 .../hbase/client/TestTableFavoredNodes.java     |     7 +-
 .../coprocessor/TestIncrementTimeRange.java     |     5 +-
 .../hbase/io/encoding/TestChangingEncoding.java |     8 +-
 .../hbase/ipc/TestSimpleRpcScheduler.java       |    13 +-
 .../hbase/master/MockNoopMasterServices.java    |    14 +-
 .../hadoop/hbase/master/MockRegionServer.java   |    37 +-
 .../hbase/master/TestAssignmentListener.java    |     1 +
 .../master/TestAssignmentManagerOnCluster.java  |  1402 --
 .../hadoop/hbase/master/TestCatalogJanitor.java |     1 +
 .../master/TestDistributedLogSplitting.java     |     1 +
 .../apache/hadoop/hbase/master/TestMaster.java  |     1 +
 .../master/TestMasterBalanceThrottling.java     |     9 +-
 .../hadoop/hbase/master/TestMasterFailover.java |    19 +-
 .../hbase/master/TestMasterStatusServlet.java   |     5 +-
 .../hbase/master/TestMetaShutdownHandler.java   |     1 +
 .../hadoop/hbase/master/TestRegionState.java    |    17 +-
 .../hadoop/hbase/master/TestRegionStates.java   |   144 -
 .../hadoop/hbase/master/TestRestartCluster.java |     1 +
 .../assignment/AssignmentTestingUtil.java       |   125 +
 .../master/assignment/MockMasterServices.java   |   201 +
 .../assignment/TestAssignmentManager.java       |   567 +
 .../assignment/TestAssignmentOnRSCrash.java     |   185 +
 .../TestMergeTableRegionsProcedure.java         |   239 +
 .../master/assignment/TestRegionStates.java     |   226 +
 .../TestSplitTableRegionProcedure.java          |   427 +
 .../MasterProcedureTestingUtility.java          |    67 +-
 .../procedure/TestAddColumnFamilyProcedure.java |    34 +-
 .../procedure/TestCloneSnapshotProcedure.java   |    10 +-
 .../procedure/TestCreateNamespaceProcedure.java |     4 +-
 .../procedure/TestCreateTableProcedure.java     |    46 +-
 .../TestDeleteColumnFamilyProcedure.java        |    31 +-
 .../procedure/TestDeleteNamespaceProcedure.java |     4 +-
 .../procedure/TestDeleteTableProcedure.java     |    21 +-
 .../procedure/TestDisableTableProcedure.java    |    24 +-
 .../procedure/TestEnableTableProcedure.java     |    24 +-
 .../TestMasterFailoverWithProcedures.java       |    23 +-
 .../procedure/TestMasterProcedureEvents.java    |     2 +-
 .../TestModifyColumnFamilyProcedure.java        |     9 +-
 .../procedure/TestModifyNamespaceProcedure.java |     4 +-
 .../procedure/TestModifyTableProcedure.java     |    18 +-
 .../master/procedure/TestProcedureAdmin.java    |    13 +-
 .../procedure/TestRestoreSnapshotProcedure.java |    13 +-
 .../procedure/TestServerCrashProcedure.java     |   110 +-
 .../TestSplitTableRegionProcedure.java          |   420 -
 .../procedure/TestTableDDLProcedureBase.java    |     7 +-
 .../procedure/TestTruncateTableProcedure.java   |    11 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |     4 +-
 .../procedure/SimpleMasterProcedureManager.java |     2 +-
 .../regionserver/TestCompactSplitThread.java    |    24 +-
 .../hbase/regionserver/TestCompaction.java      |    10 +-
 .../regionserver/TestHRegionFileSystem.java     |     6 +-
 .../TestRegionMergeTransactionOnCluster.java    |    14 +-
 .../TestSplitTransactionOnCluster.java          |    76 +-
 .../regionserver/wal/TestAsyncLogRolling.java   |     9 +
 .../hbase/regionserver/wal/TestLogRolling.java  |     5 +
 .../wal/TestSecureAsyncWALReplay.java           |     5 +
 .../hbase/regionserver/wal/TestWALReplay.java   |     5 +
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |     4 +-
 .../hadoop/hbase/util/TestHBaseFsckMOB.java     |     2 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |    75 +-
 .../hbase/util/TestHBaseFsckReplicas.java       |     2 +-
 .../hadoop/hbase/util/TestHBaseFsckTwoRS.java   |    21 +-
 .../src/test/resources/log4j.properties         |     1 +
 187 files changed, 32709 insertions(+), 24483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index f00016d..a7a26a6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -24,8 +24,8 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -81,7 +81,7 @@ public class ClusterStatus extends VersionedWritable {
   private Collection<ServerName> deadServers;
   private ServerName master;
   private Collection<ServerName> backupMasters;
-  private Set<RegionState> intransition;
+  private List<RegionState> intransition;
   private String clusterId;
   private String[] masterCoprocessors;
   private Boolean balancerOn;
@@ -91,7 +91,7 @@ public class ClusterStatus extends VersionedWritable {
       final Collection<ServerName> deadServers,
       final ServerName master,
       final Collection<ServerName> backupMasters,
-      final Set<RegionState> rit,
+      final List<RegionState> rit,
       final String[] masterCoprocessors,
       final Boolean balancerOn) {
     this.hbaseVersion = hbaseVersion;
@@ -262,7 +262,7 @@ public class ClusterStatus extends VersionedWritable {
   }
 
   @InterfaceAudience.Private
-  public Set<RegionState> getRegionsInTransition() {
+  public List<RegionState> getRegionsInTransition() {
     return this.intransition;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index ee8d5fd..15bc132 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -2049,7 +2049,7 @@ public class MetaTableAccessor {
       + Bytes.toStringBinary(HConstants.MERGEB_QUALIFIER));
   }
 
-  private static Put addRegionInfo(final Put p, final HRegionInfo hri)
+  public static Put addRegionInfo(final Put p, final HRegionInfo hri)
     throws IOException {
     p.addImmutable(getCatalogFamily(), HConstants.REGIONINFO_QUALIFIER,
       hri.toByteArray());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index adf1496..135946f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1316,6 +1316,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         return stub.mergeTableRegions(controller, request);
       }
 
+      public MasterProtos.DispatchMergingRegionsResponse dispatchMergingRegions(
+          RpcController controller, MasterProtos.DispatchMergingRegionsRequest request)
+          throws ServiceException {
+        return stub.dispatchMergingRegions(controller, request);
+      }
+
       @Override
       public MasterProtos.AssignRegionResponse assignRegion(RpcController controller,
           MasterProtos.AssignRegionRequest request) throws ServiceException {
@@ -1335,6 +1341,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public MasterProtos.SplitTableRegionResponse splitRegion(RpcController controller,
+          MasterProtos.SplitTableRegionRequest request) throws ServiceException {
+        return stub.splitRegion(controller, request);
+      }
+
+      @Override
       public MasterProtos.DeleteTableResponse deleteTable(RpcController controller,
           MasterProtos.DeleteTableRequest request) throws ServiceException {
         return stub.deleteTable(controller, request);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index d70c76f..3469782 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -480,4 +480,15 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
     return stub.listReplicationPeers(controller, request);
   }
 
-}
+  @Override
+  public SplitTableRegionResponse splitRegion(RpcController controller, SplitTableRegionRequest request)
+      throws ServiceException {
+    return stub.splitRegion(controller, request);
+  }
+
+  @Override
+  public DispatchMergingRegionsResponse dispatchMergingRegions(RpcController controller,
+      DispatchMergingRegionsRequest request) throws ServiceException {
+    return stub.dispatchMergingRegions(controller, request);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
index c6ba030..0dd8e64 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ServerTooBusyException.java
@@ -25,14 +25,13 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
- * Throw this in rpc call if there are too many pending requests for one region server
+ * Throw this in RPC call if there are too many pending requests for one region server
  */
+@SuppressWarnings("serial")
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class ServerTooBusyException extends DoNotRetryIOException {
-
   public ServerTooBusyException(InetSocketAddress address, long count) {
-    super("There are " + count + " concurrent rpc requests for " + address);
+    super("Busy Server! " + count + " concurrent RPCs against " + address);
   }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index a930732..7116763 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -36,10 +36,8 @@ public class RegionState {
   @InterfaceStability.Evolving
   public enum State {
     OFFLINE,        // region is in an offline state
-    PENDING_OPEN,   // same as OPENING, to be removed
     OPENING,        // server has begun to open but not yet done
     OPEN,           // server opened region and updated meta
-    PENDING_CLOSE,  // same as CLOSING, to be removed
     CLOSING,        // server has begun to close but not yet done
     CLOSED,         // server closed region and updated meta
     SPLITTING,      // server started split of a region
@@ -64,18 +62,12 @@ public class RegionState {
       case OFFLINE:
         rs = ClusterStatusProtos.RegionState.State.OFFLINE;
         break;
-      case PENDING_OPEN:
-        rs = ClusterStatusProtos.RegionState.State.PENDING_OPEN;
-        break;
       case OPENING:
         rs = ClusterStatusProtos.RegionState.State.OPENING;
         break;
       case OPEN:
         rs = ClusterStatusProtos.RegionState.State.OPEN;
         break;
-      case PENDING_CLOSE:
-        rs = ClusterStatusProtos.RegionState.State.PENDING_CLOSE;
-        break;
       case CLOSING:
         rs = ClusterStatusProtos.RegionState.State.CLOSING;
         break;
@@ -124,8 +116,6 @@ public class RegionState {
         state = OFFLINE;
         break;
       case PENDING_OPEN:
-        state = PENDING_OPEN;
-        break;
       case OPENING:
         state = OPENING;
         break;
@@ -133,8 +123,6 @@ public class RegionState {
         state = OPEN;
         break;
       case PENDING_CLOSE:
-        state = PENDING_CLOSE;
-        break;
       case CLOSING:
         state = CLOSING;
         break;
@@ -166,7 +154,7 @@ public class RegionState {
         state = MERGING_NEW;
         break;
       default:
-        throw new IllegalStateException("");
+        throw new IllegalStateException("Unhandled state " + protoState);
       }
       return state;
     }
@@ -231,22 +219,16 @@ public class RegionState {
     this.ritDuration += (this.stamp - previousStamp);
   }
 
-  /**
-   * PENDING_CLOSE (to be removed) is the same as CLOSING
-   */
   public boolean isClosing() {
-    return state == State.PENDING_CLOSE || state == State.CLOSING;
+    return state == State.CLOSING;
   }
 
   public boolean isClosed() {
     return state == State.CLOSED;
   }
 
-  /**
-   * PENDING_OPEN (to be removed) is the same as OPENING
-   */
   public boolean isOpening() {
-    return state == State.PENDING_OPEN || state == State.OPENING;
+    return state == State.OPENING;
   }
 
   public boolean isOpened() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 4f68447..fcf2c34 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -1803,7 +1803,7 @@ public final class ProtobufUtil {
    * has a serialized {@link ServerName} in it.
    * @return Returns null if <code>data</code> is null else converts passed data
    * to a ServerName instance.
-   * @throws DeserializationException 
+   * @throws DeserializationException
    */
   public static ServerName toServerName(final byte [] data) throws DeserializationException {
     if (data == null || data.length <= 0) return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index f44979c..5cec10d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -20,19 +20,19 @@ package org.apache.hadoop.hbase.shaded.protobuf;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NavigableSet;
-import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
@@ -83,11 +83,13 @@ import org.apache.hadoop.hbase.io.LimitInputStream;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
 import org.apache.hadoop.hbase.quotas.QuotaType;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
@@ -101,8 +103,6 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -149,7 +149,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -166,6 +165,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
 import org.apache.hadoop.hbase.util.VersionInfo;
@@ -1814,33 +1814,6 @@ public final class ProtobufUtil {
   }
 
   /**
-   * A helper to close a region for split or merge
-   * using admin protocol.
-   *
-   * @param controller RPC controller
-   * @param admin Admin service
-   * @param server the RS that hosts the target region
-   * @param regionInfo the target region info
-   * @return true if the region is closed
-   * @throws IOException
-   */
-  public static boolean closeRegionForSplitOrMerge(
-      final RpcController controller,
-      final AdminService.BlockingInterface admin,
-      final ServerName server,
-      final HRegionInfo... regionInfo) throws IOException {
-    CloseRegionForSplitOrMergeRequest closeRegionForRequest =
-        ProtobufUtil.buildCloseRegionForSplitOrMergeRequest(server, regionInfo);
-    try {
-      CloseRegionForSplitOrMergeResponse response =
-          admin.closeRegionForSplitOrMerge(controller, closeRegionForRequest);
-      return ResponseConverter.isClosed(response);
-    } catch (ServiceException se) {
-      throw getRemoteException(se);
-    }
-  }
-
-  /**
    * A helper to warmup a region given a region name
    * using admin protocol
    *
@@ -1992,6 +1965,46 @@ public final class ProtobufUtil {
     }
   }
 
+  /**
+   * A helper to merge regions using admin protocol. Send request to
+   * regionserver.
+   * @param admin
+   * @param region_a
+   * @param region_b
+   * @param forcible true if do a compulsory merge, otherwise we will only merge
+   *          two adjacent regions
+   * @param user effective user
+   * @throws IOException
+   */
+  public static void mergeRegions(final RpcController controller,
+      final AdminService.BlockingInterface admin,
+      final HRegionInfo region_a, final HRegionInfo region_b,
+      final boolean forcible, final User user) throws IOException {
+    final MergeRegionsRequest request = ProtobufUtil.buildMergeRegionsRequest(
+        region_a.getRegionName(), region_b.getRegionName(),forcible);
+    if (user != null) {
+      try {
+        user.runAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            admin.mergeRegions(controller, request);
+            return null;
+          }
+        });
+      } catch (InterruptedException ie) {
+        InterruptedIOException iioe = new InterruptedIOException();
+        iioe.initCause(ie);
+        throw iioe;
+      }
+    } else {
+      try {
+        admin.mergeRegions(controller, request);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+    }
+  }
+
 // End helpers for Admin
 
   /*
@@ -3003,8 +3016,8 @@ public final class ProtobufUtil {
       backupMasters.add(ProtobufUtil.toServerName(sn));
     }
 
-    Set<RegionState> rit = null;
-    rit = new HashSet<>(proto.getRegionsInTransitionList().size());
+    List<RegionState> rit =
+      new ArrayList<>(proto.getRegionsInTransitionList().size());
     for (RegionInTransition region : proto.getRegionsInTransitionList()) {
       RegionState value = RegionState.convert(region.getRegionState());
       rit.add(value);
@@ -3163,26 +3176,6 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Create a CloseRegionForSplitOrMergeRequest for given regions
-   *
-   * @param server the RS server that hosts the region
-   * @param regionsToClose the info of the regions to close
-   * @return a CloseRegionForSplitRequest
-   */
-  public static CloseRegionForSplitOrMergeRequest buildCloseRegionForSplitOrMergeRequest(
-      final ServerName server,
-      final HRegionInfo... regionsToClose) {
-    CloseRegionForSplitOrMergeRequest.Builder builder =
-        CloseRegionForSplitOrMergeRequest.newBuilder();
-    for(int i = 0; i < regionsToClose.length; i++) {
-        RegionSpecifier regionToClose = RequestConverter.buildRegionSpecifier(
-          RegionSpecifierType.REGION_NAME, regionsToClose[i].getRegionName());
-        builder.addRegion(regionToClose);
-    }
-    return builder.build();
-  }
-
-  /**
     * Create a CloseRegionRequest for a given encoded region name
     *
     * @param encodedRegionName the name of the region to close
@@ -3220,6 +3213,28 @@ public final class ProtobufUtil {
      return builder.build();
    }
 
+   /**
+    * Create a MergeRegionsRequest for the given regions
+    * @param regionA name of region a
+    * @param regionB name of region b
+    * @param forcible true if it is a compulsory merge
+    * @return a MergeRegionsRequest
+    */
+   public static MergeRegionsRequest buildMergeRegionsRequest(
+       final byte[] regionA, final byte[] regionB, final boolean forcible) {
+     MergeRegionsRequest.Builder builder = MergeRegionsRequest.newBuilder();
+     RegionSpecifier regionASpecifier = RequestConverter.buildRegionSpecifier(
+         RegionSpecifierType.REGION_NAME, regionA);
+     RegionSpecifier regionBSpecifier = RequestConverter.buildRegionSpecifier(
+         RegionSpecifierType.REGION_NAME, regionB);
+     builder.setRegionA(regionASpecifier);
+     builder.setRegionB(regionBSpecifier);
+     builder.setForcible(forcible);
+     // send the master's wall clock time as well, so that the RS can refer to it
+     builder.setMasterSystemTime(EnvironmentEdgeManager.currentTime());
+     return builder.build();
+   }
+
   /**
    * Get a ServerName from the passed in data bytes.
    * @param data Data with a serialize server name in it; can handle the old style
@@ -3263,4 +3278,4 @@ public final class ProtobufUtil {
     int port = Addressing.parsePort(str);
     return ServerName.valueOf(hostname, port, -1L);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index a513d66..7b50c3f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -117,7 +117,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -1113,19 +1112,6 @@ public final class RequestConverter {
     return builder.build();
   }
 
-  public static SplitTableRegionRequest buildSplitTableRegionRequest(
-      final HRegionInfo regionInfo,
-      final byte[] splitPoint,
-      final long nonceGroup,
-      final long nonce) {
-    SplitTableRegionRequest.Builder builder = SplitTableRegionRequest.newBuilder();
-    builder.setRegionInfo(HRegionInfo.convert(regionInfo));
-    builder.setSplitRow(UnsafeByteOperations.unsafeWrap(splitPoint));
-    builder.setNonceGroup(nonceGroup);
-    builder.setNonce(nonce);
-    return builder.build();
-  }
-
   /**
    * Create a protocol buffer AssignRegionRequest
    *
@@ -1508,7 +1494,7 @@ public final class RequestConverter {
   /**
    * Create a RegionOpenInfo based on given region info and version of offline node
    */
-  private static RegionOpenInfo buildRegionOpenInfo(
+  public static RegionOpenInfo buildRegionOpenInfo(
       final HRegionInfo region,
       final List<ServerName> favoredNodes, Boolean openForReplay) {
     RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
index ecadbbc..c489628 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ResponseConverter.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.SingleResponse;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
@@ -254,18 +253,6 @@ public final class ResponseConverter {
   }
 
   /**
-   * Check if the region is closed from a CloseRegionForSplitResponse
-   *
-   * @param proto the CloseRegionForSplitResponse
-   * @return the region close state
-   */
-  public static boolean isClosed
-      (final CloseRegionForSplitOrMergeResponse proto) {
-    if (proto == null || !proto.hasClosed()) return false;
-    return proto.getClosed();
-  }
-
-  /**
    * A utility to build a GetServerInfoResponse.
    *
    * @param serverName

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index d4ec48e..19363d0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
@@ -248,7 +248,7 @@ public class ChoreService implements ChoreServicer {
    */
   static class ChoreServiceThreadFactory implements ThreadFactory {
     private final String threadPrefix;
-    private final static String THREAD_NAME_SUFFIX = "_ChoreService_";
+    private final static String THREAD_NAME_SUFFIX = "_Chore_";
     private AtomicInteger threadNumber = new AtomicInteger(1);
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index 609e9a5..3789f71 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -146,7 +146,7 @@ public final class HConstants {
   public static final int DEFAULT_HBASE_BALANCER_PERIOD = 300000;
 
   /** The name of the ensemble table */
-  public static final String ENSEMBLE_TABLE_NAME = "hbase:ensemble";
+  public static final TableName ENSEMBLE_TABLE_NAME = TableName.valueOf("hbase:ensemble");
 
   /** Config for pluggable region normalizer */
   public static final String HBASE_MASTER_NORMALIZER_CLASS =

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index 9b9755b..cba03c0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -89,7 +89,12 @@ public final class TableName implements Comparable<TableName> {
   public static final String OLD_META_STR = ".META.";
   public static final String OLD_ROOT_STR = "-ROOT-";
 
-
+  /**
+   * @return True if <code>tn</code> is the hbase:meta table name.
+   */
+  public static boolean isMetaTableName(final TableName tn) {
+    return tn.equals(TableName.META_TABLE_NAME);
+  }
 
   /**
    * TableName for old -ROOT- table. It is used to read/process old WALs which have

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
index 216fe0c..8536ce2 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKConfig.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
index f6c9cb8..7e1f836 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSource.java
@@ -46,12 +46,10 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold";
   String RIT_OLDEST_AGE_NAME = "ritOldestAge";
   String RIT_DURATION_NAME = "ritDuration";
-  String ASSIGN_TIME_NAME = "assign";
-  String BULK_ASSIGN_TIME_NAME = "bulkAssign";
-
-  void updateAssignmentTime(long time);
 
-  void updateBulkAssignTime(long time);
+  String OPERATION_COUNT_NAME = "operationCount";
+  String ASSIGN_TIME_NAME = "assign";
+  String UNASSIGN_TIME_NAME = "unassign";
 
   /**
    * Set the number of regions in transition.
@@ -75,4 +73,19 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   void setRITOldestAge(long age);
 
   void updateRitDuration(long duration);
+
+  /**
+   * Increment the count of assignment operation (assign/unassign).
+   */
+  void incrementOperationCounter();
+
+  /**
+   * Add the time took to perform the last assign operation
+   */
+  void updateAssignTime(long time);
+
+  /**
+   * Add the time took to perform the last unassign operation
+   */
+  void updateUnassignTime(long time);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
index ab504f5..722358d 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricHistogram;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
 @InterfaceAudience.Private
@@ -32,8 +33,10 @@ public class MetricsAssignmentManagerSourceImpl
   private MutableGaugeLong ritCountOverThresholdGauge;
   private MutableGaugeLong ritOldestAgeGauge;
   private MetricHistogram ritDurationHisto;
+
+  private MutableFastCounter operationCounter;
   private MetricHistogram assignTimeHisto;
-  private MetricHistogram bulkAssignTimeHisto;
+  private MetricHistogram unassignTimeHisto;
 
   public MetricsAssignmentManagerSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
@@ -49,31 +52,40 @@ public class MetricsAssignmentManagerSourceImpl
     ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
     ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
     ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
-    assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
-    bulkAssignTimeHisto = metricsRegistry.newTimeHistogram(BULK_ASSIGN_TIME_NAME);
+    operationCounter = metricsRegistry.getCounter(OPERATION_COUNT_NAME, 0l);
     ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME);
+    assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
+    unassignTimeHisto = metricsRegistry.newTimeHistogram(UNASSIGN_TIME_NAME);
   }
 
   @Override
-  public void updateAssignmentTime(long time) {
-    assignTimeHisto.add(time);
+  public void setRIT(final int ritCount) {
+    ritGauge.set(ritCount);
   }
 
   @Override
-  public void updateBulkAssignTime(long time) {
-    bulkAssignTimeHisto.add(time);
+  public void setRITCountOverThreshold(final int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
   }
 
-  public void setRIT(int ritCount) {
-    ritGauge.set(ritCount);
+  @Override
+  public void setRITOldestAge(final long ritCount) {
+    ritOldestAgeGauge.set(ritCount);
   }
 
-  public void setRITCountOverThreshold(int ritCount) {
-    ritCountOverThresholdGauge.set(ritCount);
+  @Override
+  public void incrementOperationCounter() {
+    operationCounter.incr();
   }
 
-  public void setRITOldestAge(long ritCount) {
-    ritOldestAgeGauge.set(ritCount);
+  @Override
+  public void updateAssignTime(final long time) {
+    assignTimeHisto.add(time);
+  }
+
+  @Override
+  public void updateUnassignTime(final long time) {
+    unassignTimeHisto.add(time);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
index 646bc1f..fc80c9c 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/AbstractProcedureScheduler.java
@@ -25,13 +25,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
+import com.google.common.annotations.VisibleForTesting;
 
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   private static final Log LOG = LogFactory.getLog(AbstractProcedureScheduler.class);
-
   private final ReentrantLock schedLock = new ReentrantLock();
   private final Condition schedWaitCond = schedLock.newCondition();
   private boolean running = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
index 19ba28c..e11c23c 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockAndQueue.java
@@ -19,24 +19,25 @@
 package org.apache.hadoop.hbase.procedure2;
 
 /**
- * Locking for mutual exclusion between procedures. Only by procedure framework internally.
+ * Locking for mutual exclusion between procedures. Used only by procedure framework internally.
  * {@link LockAndQueue} has two purposes:
  * <ol>
- *   <li>Acquire/release exclusive/shared locks</li>
- *   <li>Maintain a list of procedures waiting for this lock<br>
- *      To do so, {@link LockAndQueue} extends {@link ProcedureDeque} class. Using inheritance over
- *      composition for this need is unusual, but the choice is motivated by million regions
- *      assignment case as it will reduce memory footprint and number of objects to be GCed.
+ *   <li>Acquire/release exclusive/shared locks.</li>
+ *   <li>Maintains a list of procedures waiting on this lock.
+ *      {@link LockAndQueue} extends {@link ProcedureDeque} class. Blocked Procedures are added
+ *      to our super Deque. Using inheritance over composition to keep the Deque of waiting
+ *      Procedures is unusual, but we do it this way because in certain cases, there will be
+ *      millions of regions. This layout uses less memory.
  * </ol>
  *
- * NOT thread-safe. Needs external concurrency control. For eg. Uses in MasterProcedureScheduler are
+ * <p>NOT thread-safe. Needs external concurrency control: e.g. uses in MasterProcedureScheduler are
  * guarded by schedLock().
  * <br>
  * There is no need of 'volatile' keyword for member variables because of memory synchronization
  * guarantees of locks (see 'Memory Synchronization',
  * http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/Lock.html)
  * <br>
- * We do not implement Lock interface because we need exclusive + shared locking, and also
+ * We do not implement Lock interface because we need exclusive and shared locking, and also
  * because try-lock functions require procedure id.
  * <br>
  * We do not use ReentrantReadWriteLock directly because of its high memory overhead.
@@ -104,6 +105,9 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
     return true;
   }
 
+  /**
+   * @return True if we released a lock.
+   */
   public boolean releaseExclusiveLock(final Procedure proc) {
     if (isLockOwner(proc.getProcId())) {
       exclusiveLockProcIdOwner = Long.MIN_VALUE;
@@ -111,4 +115,11 @@ public class LockAndQueue extends ProcedureDeque implements LockStatus {
     }
     return false;
   }
-}
+
+  @Override
+  public String toString() {
+    return "exclusiveLockOwner=" + (hasExclusiveLock()? getExclusiveLockProcIdOwner(): "NONE") +
+      ", sharedLockCount=" + getSharedLockCount() +
+      ", waitingProcCount=" + size();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index fee5250..2841697 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -253,9 +253,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
    */
   protected StringBuilder toStringSimpleSB() {
     final StringBuilder sb = new StringBuilder();
-    toStringClassDetails(sb);
 
-    sb.append(", procId=");
+    sb.append("procId=");
     sb.append(getProcId());
 
     if (hasParent()) {
@@ -275,6 +274,9 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
       sb.append(", failed=" + getException());
     }
 
+    sb.append(", ");
+    toStringClassDetails(sb);
+
     return sb;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
index cb90ac0..43cce3a 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureEvent.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.procedure2;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Basic ProcedureEvent that contains an "object", which can be a description or a reference to the
@@ -50,6 +49,7 @@ public class ProcedureEvent<T> {
 
   @Override
   public String toString() {
-    return getClass().getSimpleName() + "(" + object + ")";
+    return getClass().getSimpleName() + " for " + object + ", ready=" + isReady() +
+        ", suspended procedures count=" + getSuspendedProcedures().size();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 0856aa2..3145e83 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -360,8 +360,7 @@ public class ProcedureExecutor<TEnvironment> {
       assert !(proc.isFinished() && !proc.hasParent()) : "unexpected completed proc=" + proc;
 
       if (debugEnabled) {
-        LOG.debug(String.format("Loading state=%s isFailed=%s: %s",
-                    proc.getState(), proc.hasException(), proc));
+        LOG.debug(String.format("Loading %s", proc));
       }
 
       Long rootProcId = getRootProcedureId(proc);
@@ -483,7 +482,7 @@ public class ProcedureExecutor<TEnvironment> {
     // We have numThreads executor + one timer thread used for timing out
     // procedures and triggering periodic procedures.
     this.corePoolSize = numThreads;
-    LOG.info("Starting executor threads=" + corePoolSize);
+    LOG.info("Starting executor worker threads=" + corePoolSize);
 
     // Create the Thread Group for the executors
     threadGroup = new ThreadGroup("ProcedureExecutor");
@@ -522,7 +521,9 @@ public class ProcedureExecutor<TEnvironment> {
       store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
 
     // Start the executors. Here we must have the lastProcId set.
-    LOG.debug("Start workers " + workerThreads.size());
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Start workers " + workerThreads.size());
+    }
     timeoutExecutor.start();
     for (WorkerThread worker: workerThreads) {
       worker.start();
@@ -1147,8 +1148,7 @@ public class ProcedureExecutor<TEnvironment> {
 
       if (proc.isSuccess()) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Completed in " +
-              StringUtils.humanTimeDiff(proc.elapsedTime()) + ": " + proc);
+          LOG.debug("Completed " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
         }
         // Finalize the procedure state
         if (proc.getProcId() == rootProcId) {
@@ -1342,7 +1342,7 @@ public class ProcedureExecutor<TEnvironment> {
         return;
       } catch (Throwable e) {
         // Catch NullPointerExceptions or similar errors...
-        String msg = "CODE-BUG: Uncatched runtime exception for procedure: " + procedure;
+        String msg = "CODE-BUG: Uncaught runtime exception: " + procedure;
         LOG.error(msg, e);
         procedure.setFailure(new RemoteProcedureException(msg, e));
       }
@@ -1674,7 +1674,7 @@ public class ProcedureExecutor<TEnvironment> {
         // if the procedure is in a waiting state again, put it back in the queue
         procedure.updateTimestamp();
         if (procedure.isWaiting()) {
-          delayed.setTimeoutTimestamp(procedure.getTimeoutTimestamp());
+          delayed.setTimeout(procedure.getTimeoutTimestamp());
           queue.add(delayed);
         }
       } else {
@@ -1752,7 +1752,7 @@ public class ProcedureExecutor<TEnvironment> {
     }
 
     @Override
-    public long getTimeoutTimestamp() {
+    public long getTimeout() {
       return timeout;
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
index 16ff781..617532b 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
@@ -23,13 +23,11 @@ import com.google.common.annotations.VisibleForTesting;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Keep track of the runnable procedures
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public interface ProcedureScheduler {
   /**
    * Start the scheduler
@@ -93,7 +91,7 @@ public interface ProcedureScheduler {
   Procedure poll(long timeout, TimeUnit unit);
 
   /**
-   * Mark the event has not ready.
+   * Mark the event as not ready.
    * procedures calling waitEvent() will be suspended.
    * @param event the event to mark as suspended/not ready
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
new file mode 100644
index 0000000..0e33887
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -0,0 +1,367 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedContainerWithTimestamp;
+import org.apache.hadoop.hbase.procedure2.util.DelayedUtil.DelayedWithTimeout;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
+
+import com.google.common.collect.ArrayListMultimap;
+
+/**
+ * A procedure dispatcher that aggregates and sends after elapsed time or after we hit
+ * count threshold. Creates its own threadpool to run RPCs with timeout.
+ * <ul>
+ * <li>Each server queue has a dispatch buffer</li>
+ * <li>Once the dispatch buffer reaches a threshold-size/time we send<li>
+ * </ul>
+ * <p>Call {@link #start()} and then {@link #submitTask(Callable)}. When done,
+ * call {@link #stop()}.
+ */
+@InterfaceAudience.Private
+public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable<TRemote>> {
+  private static final Log LOG = LogFactory.getLog(RemoteProcedureDispatcher.class);
+
+  public static final String THREAD_POOL_SIZE_CONF_KEY =
+      "hbase.procedure.remote.dispatcher.threadpool.size";
+  private static final int DEFAULT_THREAD_POOL_SIZE = 128;
+
+  public static final String DISPATCH_DELAY_CONF_KEY =
+      "hbase.procedure.remote.dispatcher.delay.msec";
+  private static final int DEFAULT_DISPATCH_DELAY = 150;
+
+  public static final String DISPATCH_MAX_QUEUE_SIZE_CONF_KEY =
+      "hbase.procedure.remote.dispatcher.max.queue.size";
+  private static final int DEFAULT_MAX_QUEUE_SIZE = 32;
+
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final ConcurrentHashMap<TRemote, BufferNode> nodeMap =
+      new ConcurrentHashMap<TRemote, BufferNode>();
+
+  private final int operationDelay;
+  private final int queueMaxSize;
+  private final int corePoolSize;
+
+  private TimeoutExecutorThread timeoutExecutor;
+  private ThreadPoolExecutor threadPool;
+
+  protected RemoteProcedureDispatcher(Configuration conf) {
+    this.corePoolSize = conf.getInt(THREAD_POOL_SIZE_CONF_KEY, DEFAULT_THREAD_POOL_SIZE);
+    this.operationDelay = conf.getInt(DISPATCH_DELAY_CONF_KEY, DEFAULT_DISPATCH_DELAY);
+    this.queueMaxSize = conf.getInt(DISPATCH_MAX_QUEUE_SIZE_CONF_KEY, DEFAULT_MAX_QUEUE_SIZE);
+  }
+
+  public boolean start() {
+    if (running.getAndSet(true)) {
+      LOG.warn("Already running");
+      return false;
+    }
+
+    LOG.info("Starting procedure remote dispatcher; threads=" + this.corePoolSize +
+      ", queueMaxSize=" + this.queueMaxSize + ", operationDelay=" + this.operationDelay);
+
+    // Create the timeout executor
+    timeoutExecutor = new TimeoutExecutorThread();
+    timeoutExecutor.start();
+
+    // Create the thread pool that will execute RPCs
+    threadPool = Threads.getBoundedCachedThreadPool(corePoolSize, 60L, TimeUnit.SECONDS,
+      Threads.newDaemonThreadFactory("ProcedureRemoteDispatcher", getUncaughtExceptionHandler()));
+    return true;
+  }
+
+  public boolean stop() {
+    if (!running.getAndSet(false)) {
+      return false;
+    }
+
+    LOG.info("Stopping procedure remote dispatcher");
+
+    // send stop signals
+    timeoutExecutor.sendStopSignal();
+    threadPool.shutdownNow();
+    return true;
+  }
+
+  public void join() {
+    assert !running.get() : "expected not running";
+
+    // wait the timeout executor
+    timeoutExecutor.awaitTermination();
+    timeoutExecutor = null;
+
+    // wait for the thread pool to terminate
+    threadPool.shutdownNow();
+    try {
+      while (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
+        LOG.warn("Waiting for thread-pool to terminate");
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting for thread-pool termination", e);
+    }
+  }
+
+  protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
+    return new UncaughtExceptionHandler() {
+      @Override
+      public void uncaughtException(Thread t, Throwable e) {
+        LOG.warn("Failed to execute remote procedures " + t.getName(), e);
+      }
+    };
+  }
+
+  // ============================================================================================
+  //  Node Helpers
+  // ============================================================================================
+  /**
+   * Add a node that will be able to execute remove procedures
+   * @param key the node identifier
+   */
+  public void addNode(final TRemote key) {
+    final BufferNode newNode = new BufferNode(key);
+    nodeMap.putIfAbsent(key, newNode);
+  }
+
+  /**
+   * Get the remote node that will execute remote procedures
+   * @param key the node identifier
+   */
+  public RemoteNode getNode(final TRemote key) {
+    assert key != null : "found null key for node";
+    return nodeMap.get(key);
+  }
+
+  /**
+   * Remove a remote node
+   * @param key the node identifier
+   */
+  public boolean removeNode(final TRemote key) {
+    final BufferNode node = nodeMap.remove(key);
+    if (node == null) return false;
+    node.abortOperationsInQueue();
+    return true;
+  }
+
+  // ============================================================================================
+  //  Task Helpers
+  // ============================================================================================
+  protected Future<Void> submitTask(Callable<Void> task) {
+    return threadPool.submit(task);
+  }
+
+  protected Future<Void> submitTask(Callable<Void> task, long delay, TimeUnit unit) {
+    final FutureTask<Void> futureTask = new FutureTask(task);
+    timeoutExecutor.add(new DelayedTask(futureTask, delay, unit));
+    return futureTask;
+  }
+
+  protected abstract void remoteDispatch(TRemote key, Set<RemoteProcedure> operations);
+  protected abstract void abortPendingOperations(TRemote key, Set<RemoteProcedure> operations);
+
+  /**
+   * Data structure with reference to remote operation.
+   */
+  public static abstract class RemoteOperation {
+    private final RemoteProcedure remoteProcedure;
+
+    protected RemoteOperation(final RemoteProcedure remoteProcedure) {
+      this.remoteProcedure = remoteProcedure;
+    }
+
+    public RemoteProcedure getRemoteProcedure() {
+      return remoteProcedure;
+    }
+  }
+
+  /**
+   * Remote procedure reference.
+   * @param <TEnv>
+   * @param <TRemote>
+   */
+  public interface RemoteProcedure<TEnv, TRemote> {
+    RemoteOperation remoteCallBuild(TEnv env, TRemote remote);
+    void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response);
+    void remoteCallFailed(TEnv env, TRemote remote, IOException exception);
+  }
+
+  /**
+   * Account of what procedures are running on remote node.
+   * @param <TEnv>
+   * @param <TRemote>
+   */
+  public interface RemoteNode<TEnv, TRemote> {
+    TRemote getKey();
+    void add(RemoteProcedure<TEnv, TRemote> operation);
+    void dispatch();
+  }
+
+  protected ArrayListMultimap<Class<?>, RemoteOperation> buildAndGroupRequestByType(final TEnv env,
+      final TRemote remote, final Set<RemoteProcedure> operations) {
+    final ArrayListMultimap<Class<?>, RemoteOperation> requestByType = ArrayListMultimap.create();
+    for (RemoteProcedure proc: operations) {
+      RemoteOperation operation = proc.remoteCallBuild(env, remote);
+      requestByType.put(operation.getClass(), operation);
+    }
+    return requestByType;
+  }
+
+  protected <T extends RemoteOperation> List<T> fetchType(
+      final ArrayListMultimap<Class<?>, RemoteOperation> requestByType, final Class<T> type) {
+    return (List<T>)requestByType.removeAll(type);
+  }
+
+  // ============================================================================================
+  //  Timeout Helpers
+  // ============================================================================================
+  private final class TimeoutExecutorThread extends Thread {
+    private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<DelayedWithTimeout>();
+
+    public TimeoutExecutorThread() {
+      super("ProcedureDispatcherTimeoutThread");
+    }
+
+    @Override
+    public void run() {
+      while (running.get()) {
+        final DelayedWithTimeout task = DelayedUtil.takeWithoutInterrupt(queue);
+        if (task == null || task == DelayedUtil.DELAYED_POISON) {
+          // the executor may be shutting down, and the task is just the shutdown request
+          continue;
+        }
+
+        if (task instanceof DelayedTask) {
+          threadPool.execute(((DelayedTask)task).getObject());
+        } else {
+          ((BufferNode)task).dispatch();
+        }
+      }
+    }
+
+    public void add(final DelayedWithTimeout delayed) {
+      queue.add(delayed);
+    }
+
+    public void remove(final DelayedWithTimeout delayed) {
+      queue.remove(delayed);
+    }
+
+    public void sendStopSignal() {
+      queue.add(DelayedUtil.DELAYED_POISON);
+    }
+
+    public void awaitTermination() {
+      try {
+        final long startTime = EnvironmentEdgeManager.currentTime();
+        for (int i = 0; isAlive(); ++i) {
+          sendStopSignal();
+          join(250);
+          if (i > 0 && (i % 8) == 0) {
+            LOG.warn("Waiting termination of thread " + getName() + ", " +
+              StringUtils.humanTimeDiff(EnvironmentEdgeManager.currentTime() - startTime));
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.warn(getName() + " join wait got interrupted", e);
+      }
+    }
+  }
+
+  // ============================================================================================
+  //  Internals Helpers
+  // ============================================================================================
+
+  /**
+   * Node that contains a set of RemoteProcedures
+   */
+  protected final class BufferNode extends DelayedContainerWithTimestamp<TRemote>
+      implements RemoteNode<TEnv, TRemote> {
+    private Set<RemoteProcedure> operations;
+
+    protected BufferNode(final TRemote key) {
+      super(key, 0);
+    }
+
+    public TRemote getKey() {
+      return getObject();
+    }
+
+    public synchronized void add(final RemoteProcedure operation) {
+      if (this.operations == null) {
+        this.operations = new HashSet<>();
+        setTimeout(EnvironmentEdgeManager.currentTime() + operationDelay);
+        timeoutExecutor.add(this);
+      }
+      this.operations.add(operation);
+      if (this.operations.size() > queueMaxSize) {
+        timeoutExecutor.remove(this);
+        dispatch();
+      }
+    }
+
+    public synchronized void dispatch() {
+      if (operations != null) {
+        remoteDispatch(getKey(), operations);
+        this.operations = null;
+      }
+    }
+
+    public synchronized void abortOperationsInQueue() {
+      if (operations != null) {
+        abortPendingOperations(getKey(), operations);
+        this.operations = null;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return super.toString() + ", operations=" + this.operations;
+    }
+  }
+
+  /**
+   * Delayed object that holds a FutureTask.
+   * used to submit something later to the thread-pool.
+   */
+  private static final class DelayedTask extends DelayedContainerWithTimestamp<FutureTask<Void>> {
+    public DelayedTask(final FutureTask<Void> task, final long delay, final TimeUnit unit) {
+      super(task, EnvironmentEdgeManager.currentTime() + unit.toMillis(delay));
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index 5c3a4c7..ea2a41f 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -108,6 +108,9 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
     if (aborted.get() && isRollbackSupported(getCurrentState())) {
       setAbortFailure(getClass().getSimpleName(), "abort requested");
     } else {
+      if (aborted.get()) {
+        LOG.warn("ignoring abort request " + state);
+      }
       setNextState(getStateId(state));
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
index ea34c49..3719019 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/DelayedUtil.java
@@ -32,13 +32,19 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 public final class DelayedUtil {
   private DelayedUtil() { }
 
+  /**
+   * Add a timeout to a Delay
+   */
   public interface DelayedWithTimeout extends Delayed {
-    long getTimeoutTimestamp();
+    long getTimeout();
   }
 
+  /**
+   * POISON implementation; used to mark special state: e.g. shutdown.
+   */
   public static final DelayedWithTimeout DELAYED_POISON = new DelayedWithTimeout() {
     @Override
-    public long getTimeoutTimestamp() {
+    public long getTimeout() {
       return 0;
     }
 
@@ -49,7 +55,7 @@ public final class DelayedUtil {
 
     @Override
     public int compareTo(final Delayed o) {
-      return Long.compare(0, DelayedUtil.getTimeoutTimestamp(o));
+      return Long.compare(0, DelayedUtil.getTimeout(o));
     }
 
     @Override
@@ -59,10 +65,13 @@ public final class DelayedUtil {
 
     @Override
     public String toString() {
-      return getClass().getSimpleName() + "(POISON)";
+      return "POISON";
     }
   };
 
+  /**
+   * @return null (if an interrupt) or an instance of E; resets interrupt on calling thread.
+   */
   public static <E extends Delayed> E takeWithoutInterrupt(final DelayQueue<E> queue) {
     try {
       return queue.take();
@@ -72,33 +81,42 @@ public final class DelayedUtil {
     }
   }
 
-  public static long getRemainingTime(final TimeUnit resultUnit, final long timeoutTime) {
+  /**
+   * @return Time remaining as milliseconds.
+   */
+  public static long getRemainingTime(final TimeUnit resultUnit, final long timeout) {
     final long currentTime = EnvironmentEdgeManager.currentTime();
-    if (currentTime >= timeoutTime) {
+    if (currentTime >= timeout) {
       return 0;
     }
-    return resultUnit.convert(timeoutTime - currentTime, TimeUnit.MILLISECONDS);
+    return resultUnit.convert(timeout - currentTime, TimeUnit.MILLISECONDS);
   }
 
   public static int compareDelayed(final Delayed o1, final Delayed o2) {
-    return Long.compare(getTimeoutTimestamp(o1), getTimeoutTimestamp(o2));
+    return Long.compare(getTimeout(o1), getTimeout(o2));
   }
 
-  private static long getTimeoutTimestamp(final Delayed o) {
+  private static long getTimeout(final Delayed o) {
     assert o instanceof DelayedWithTimeout : "expected DelayedWithTimeout instance, got " + o;
-    return ((DelayedWithTimeout)o).getTimeoutTimestamp();
+    return ((DelayedWithTimeout)o).getTimeout();
   }
 
   public static abstract class DelayedObject implements DelayedWithTimeout {
     @Override
     public long getDelay(final TimeUnit unit) {
-      return DelayedUtil.getRemainingTime(unit, getTimeoutTimestamp());
+      return DelayedUtil.getRemainingTime(unit, getTimeout());
     }
 
     @Override
     public int compareTo(final Delayed other) {
       return DelayedUtil.compareDelayed(this, other);
     }
+
+    @Override
+    public String toString() {
+      long timeout = getTimeout();
+      return "timeout=" + timeout + ", delay=" + getDelay(TimeUnit.MILLISECONDS);
+    }
   }
 
   public static abstract class DelayedContainer<T> extends DelayedObject {
@@ -126,25 +144,25 @@ public final class DelayedUtil {
 
     @Override
     public String toString() {
-      return getClass().getSimpleName() + "(" + getObject() + ")";
+      return "containedObject=" + getObject() + ", " + super.toString();
     }
   }
 
   public static class DelayedContainerWithTimestamp<T> extends DelayedContainer<T> {
-    private long timeoutTimestamp;
+    private long timeout;
 
-    public DelayedContainerWithTimestamp(final T object, final long timeoutTimestamp) {
+    public DelayedContainerWithTimestamp(final T object, final long timeout) {
       super(object);
-      setTimeoutTimestamp(timeoutTimestamp);
+      setTimeout(timeout);
     }
 
     @Override
-    public long getTimeoutTimestamp() {
-      return timeoutTimestamp;
+    public long getTimeout() {
+      return timeout;
     }
 
-    public void setTimeoutTimestamp(final long timeoutTimestamp) {
-      this.timeoutTimestamp = timeoutTimestamp;
+    public void setTimeout(final long timeout) {
+      this.timeout = timeout;
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 226666f..0240465 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.hbase.procedure2;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.concurrent.Callable;
 import java.util.ArrayList;
 import java.util.Set;
+import java.util.concurrent.Callable;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,18 +39,14 @@ import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
+import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
-import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Threads;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class ProcedureTestingUtility {
   private static final Log LOG = LogFactory.getLog(ProcedureTestingUtility.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java
index a2cd70f..019b456 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestDelayedUtil.java
@@ -80,7 +80,7 @@ public class TestDelayedUtil {
     }
 
     @Override
-    public long getTimeoutTimestamp() {
+    public long getTimeout() {
       return 0;
     }
   }


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
new file mode 100644
index 0000000..0869c40
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -0,0 +1,315 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+/**
+ * Base class for the Assign and Unassign Procedure.
+ * There can only be one RegionTransitionProcedure per region running at a time
+ * since each procedure takes a lock on the region (see MasterProcedureScheduler).
+ *
+ * <p>This procedure is asynchronous and responds to external events.
+ * The AssignmentManager will notify this procedure when the RS completes
+ * the operation and reports the transitioned state
+ * (see the Assign and Unassign class for more details).
+ */
+@InterfaceAudience.Private
+public abstract class RegionTransitionProcedure
+    extends Procedure<MasterProcedureEnv>
+    implements TableProcedureInterface,
+      RemoteProcedure<MasterProcedureEnv, ServerName> {
+  private static final Log LOG = LogFactory.getLog(RegionTransitionProcedure.class);
+
+  protected final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private RegionTransitionState transitionState =
+      RegionTransitionState.REGION_TRANSITION_QUEUE;
+  private RegionStateNode regionNode = null;
+  private HRegionInfo regionInfo;
+  private boolean hasLock = false;
+
+  public RegionTransitionProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public RegionTransitionProcedure(final HRegionInfo regionInfo) {
+    this.regionInfo = regionInfo;
+  }
+
+  public HRegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  protected void setRegionInfo(final HRegionInfo regionInfo) {
+    // Setter is for deserialization.
+    this.regionInfo = regionInfo;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return getRegionInfo().getTable();
+  }
+
+  public boolean isMeta() {
+    return TableName.isMetaTableName(getTableName());
+  }
+
+  @Override
+  public void toStringClassDetails(final StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" table=");
+    sb.append(getTableName());
+    sb.append(", region=");
+    sb.append(getRegionInfo().getEncodedName());
+    if (regionNode != null) {
+      sb.append(", server=");
+      sb.append(regionNode.getRegionLocation());
+    }
+  }
+
+  public RegionStateNode getRegionState(final MasterProcedureEnv env) {
+    if (regionNode == null) {
+      regionNode = env.getAssignmentManager()
+        .getRegionStates().getOrCreateRegionNode(getRegionInfo());
+    }
+    return regionNode;
+  }
+
+  protected void setTransitionState(final RegionTransitionState state) {
+    this.transitionState = state;
+  }
+
+  protected RegionTransitionState getTransitionState() {
+    return transitionState;
+  }
+
+  protected abstract boolean startTransition(MasterProcedureEnv env, RegionStateNode regionNode)
+    throws IOException, ProcedureSuspendedException;
+  protected abstract boolean updateTransition(MasterProcedureEnv env, RegionStateNode regionNode)
+    throws IOException, ProcedureSuspendedException;
+  protected abstract void completeTransition(MasterProcedureEnv env, RegionStateNode regionNode)
+    throws IOException, ProcedureSuspendedException;
+
+  protected abstract void reportTransition(MasterProcedureEnv env,
+      RegionStateNode regionNode, TransitionCode code, long seqId) throws UnexpectedStateException;
+
+  public abstract RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName serverName);
+  protected abstract void remoteCallFailed(MasterProcedureEnv env,
+      RegionStateNode regionNode, IOException exception);
+
+  @Override
+  public void remoteCallCompleted(final MasterProcedureEnv env,
+      final ServerName serverName, final RemoteOperation response) {
+    // Ignore the response? reportTransition() is the one that count?
+  }
+
+  @Override
+  public void remoteCallFailed(final MasterProcedureEnv env,
+      final ServerName serverName, final IOException exception) {
+    final RegionStateNode regionNode = getRegionState(env);
+    assert serverName.equals(regionNode.getRegionLocation()); // TODO
+    LOG.warn("Remote call failed " + regionNode + ": " + exception.getMessage());
+    remoteCallFailed(env, regionNode, exception);
+    env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
+  }
+
+  protected void addToRemoteDispatcher(final MasterProcedureEnv env,
+      final ServerName targetServer) {
+    assert targetServer.equals(getRegionState(env).getRegionLocation()) :
+      "targetServer=" + targetServer + " getRegionLocation=" + getRegionState(env).getRegionLocation(); // TODO
+
+    LOG.info("ADD TO REMOTE DISPATCHER " + getRegionState(env) + ": " + targetServer);
+
+    // Add the open region operation to the server dispatch queue.
+    // The pending close will be dispatched to the server together with the other
+    // pending operation for that server.
+    env.getProcedureScheduler().suspendEvent(getRegionState(env).getProcedureEvent());
+
+    // TODO: If the server is gone... go on failure/retry
+    env.getRemoteDispatcher().getNode(targetServer).add(this);
+  }
+
+  protected void reportTransition(final MasterProcedureEnv env, final ServerName serverName,
+      final TransitionCode code, final long seqId) throws UnexpectedStateException {
+    final RegionStateNode regionNode = getRegionState(env);
+    if (!serverName.equals(regionNode.getRegionLocation())) {
+      if (isMeta() && regionNode.getRegionLocation() == null) {
+        regionNode.setRegionLocation(serverName);
+      } else {
+        throw new UnexpectedStateException(String.format(
+          "reported unexpected transition state=%s from server=%s on region=%s, expected server=%s",
+          code, serverName, regionNode.getRegionInfo(), regionNode.getRegionLocation()));
+      }
+    }
+
+    reportTransition(env, regionNode, code, seqId);
+    env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
+  }
+
+  protected boolean isServerOnline(final MasterProcedureEnv env, final RegionStateNode regionNode) {
+    return isServerOnline(env, regionNode.getRegionLocation());
+  }
+
+  protected boolean isServerOnline(final MasterProcedureEnv env, final ServerName serverName) {
+    return env.getMasterServices().getServerManager().isServerOnline(serverName);
+  }
+
+  @Override
+  protected Procedure[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
+    final AssignmentManager am = env.getAssignmentManager();
+    final RegionStateNode regionNode = getRegionState(env);
+    LOG.debug("" + transitionState + " " + this);
+    if (!am.addRegionInTransition(regionNode, this)) {
+      String msg = String.format(
+        "There is already another procedure running on this region this=%s owner=%s",
+        this, regionNode.getProcedure());
+      LOG.warn(msg);
+      setAbortFailure(getClass().getSimpleName(), msg);
+      return null;
+    }
+    try {
+      boolean retry;
+      do {
+        retry = false;
+        switch (transitionState) {
+          case REGION_TRANSITION_QUEUE:
+            // 1. push into the AM queue for balancer policy
+            if (!startTransition(env, regionNode)) {
+              // the operation aborted, check getException()
+              am.removeRegionInTransition(getRegionState(env), this);
+              return null;
+            }
+            transitionState = RegionTransitionState.REGION_TRANSITION_DISPATCH;
+            if (env.getProcedureScheduler().waitEvent(regionNode.getProcedureEvent(), this)) {
+              throw new ProcedureSuspendedException();
+            }
+            break;
+
+          case REGION_TRANSITION_DISPATCH:
+            // 2. send the request to the target server
+            if (!updateTransition(env, regionNode)) {
+              // The operation aborted, check getException()
+              am.removeRegionInTransition(regionNode, this);
+              return null;
+            }
+            if (transitionState != RegionTransitionState.REGION_TRANSITION_DISPATCH) {
+              retry = true;
+              break;
+            }
+            if (env.getProcedureScheduler().waitEvent(regionNode.getProcedureEvent(), this)) {
+              throw new ProcedureSuspendedException();
+            }
+            break;
+
+          case REGION_TRANSITION_FINISH:
+            // 3. wait assignment response. completion/failure
+            completeTransition(env, regionNode);
+            am.removeRegionInTransition(regionNode, this);
+            return null;
+        }
+      } while (retry);
+    } catch (IOException e) {
+      LOG.warn("Retryable error trying to transition: " + regionNode, e);
+    }
+
+    return new Procedure[] { this };
+  }
+
+  @Override
+  protected void rollback(final MasterProcedureEnv env) {
+    if (isRollbackSupported(transitionState)) {
+      // Nothing done up to this point. abort safely.
+      // This should happen when something like disableTable() is triggered.
+      env.getAssignmentManager().removeRegionInTransition(regionNode, this);
+      return;
+    }
+
+    // There is no rollback for assignment unless we cancel the operation by
+    // dropping/disabling the table.
+    throw new UnsupportedOperationException("unhandled state " + transitionState);
+  }
+
+  protected abstract boolean isRollbackSupported(final RegionTransitionState state);
+
+  @Override
+  protected boolean abort(final MasterProcedureEnv env) {
+    if (isRollbackSupported(transitionState)) {
+      aborted.set(true);
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    // Unless we are assigning meta, wait for meta to be available and loaded.
+    if (!isMeta() && (env.waitFailoverCleanup(this) ||
+        env.getAssignmentManager().waitMetaInitialized(this, getRegionInfo()))) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+
+    // TODO: Revisit this and move it to the executor
+    if (env.getProcedureScheduler().waitRegion(this, getRegionInfo())) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    this.hasLock = true;
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakeRegion(this, getRegionInfo());
+    hasLock = false;
+  }
+
+  protected boolean holdLock(final MasterProcedureEnv env) {
+    return true;
+  }
+
+  protected boolean hasLock(final MasterProcedureEnv env) {
+    return hasLock;
+  }
+
+  @Override
+  protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
+    // The operation is triggered internally on the server
+    // the client does not know about this procedure.
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
new file mode 100644
index 0000000..3bb4e78
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -0,0 +1,731 @@
+/**
+ * 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.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * The procedure to split a region in a table.
+ */
+@InterfaceAudience.Private
+public class SplitTableRegionProcedure
+    extends AbstractStateMachineTableProcedure<SplitTableRegionState> {
+  private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
+
+  private Boolean traceEnabled = null;
+
+  private HRegionInfo parentHRI;
+  private HRegionInfo daughter_1_HRI;
+  private HRegionInfo daughter_2_HRI;
+
+  public SplitTableRegionProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public SplitTableRegionProcedure(final MasterProcedureEnv env,
+      final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
+    super(env);
+
+    checkSplitRow(regionToSplit, splitRow);
+
+    this.parentHRI = regionToSplit;
+
+    final TableName table = regionToSplit.getTable();
+    final long rid = getDaughterRegionIdTimestamp(regionToSplit);
+    this.daughter_1_HRI = new HRegionInfo(table, regionToSplit.getStartKey(), splitRow, false, rid);
+    this.daughter_2_HRI = new HRegionInfo(table, splitRow, regionToSplit.getEndKey(), false, rid);
+  }
+
+  private static void checkSplitRow(final HRegionInfo regionToSplit, final byte[] splitRow)
+      throws IOException {
+    if (splitRow == null || splitRow.length == 0) {
+      throw new DoNotRetryIOException("Split row cannot be null");
+    }
+
+    if (Bytes.equals(regionToSplit.getStartKey(), splitRow)) {
+      throw new DoNotRetryIOException(
+        "Split row is equal to startkey: " + Bytes.toStringBinary(splitRow));
+    }
+
+    if (!regionToSplit.containsRow(splitRow)) {
+      throw new DoNotRetryIOException(
+        "Split row is not inside region key range splitKey:" + Bytes.toStringBinary(splitRow) +
+        " region: " + regionToSplit);
+    }
+  }
+
+  /**
+   * Calculate daughter regionid to use.
+   * @param hri Parent {@link HRegionInfo}
+   * @return Daughter region id (timestamp) to use.
+   */
+  private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
+    long rid = EnvironmentEdgeManager.currentTime();
+    // Regionid is timestamp.  Can't be less than that of parent else will insert
+    // at wrong location in hbase:meta (See HBASE-710).
+    if (rid < hri.getRegionId()) {
+      LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
+        " but current time here is " + rid);
+      rid = hri.getRegionId() + 1;
+    }
+    return rid;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final SplitTableRegionState state)
+      throws InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case SPLIT_TABLE_REGION_PREPARE:
+        if (prepareSplitRegion(env)) {
+          setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION);
+          break;
+        } else {
+          assert isFailed() : "split region should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
+      case SPLIT_TABLE_REGION_PRE_OPERATION:
+        preSplitRegion(env);
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION);
+        break;
+      case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
+        addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
+        break;
+      case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
+        createDaughterRegions(env);
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR);
+        break;
+      case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
+        preSplitRegionBeforePONR(env);
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_UPDATE_META);
+        break;
+      case SPLIT_TABLE_REGION_UPDATE_META:
+        // This is the point of no return.  Adding subsequent edits to .META. as we
+        // do below when we do the daughter opens adding each to .META. can fail in
+        // various interesting ways the most interesting of which is a timeout
+        // BUT the edits all go through (See HBASE-3872).  IF we reach the PONR
+        // then subsequent failures need to crash out this region server; the
+        // server shutdown processing should be able to fix-up the incomplete split.
+        // The offlined parent will have the daughters as extra columns.  If
+        // we leave the daughter regions in place and do not remove them when we
+        // crash out, then they will have their references to the parent in place
+        // still and the server shutdown fixup of .META. will point to these
+        // regions.
+        // We should add PONR JournalEntry before offlineParentInMeta,so even if
+        // OfflineParentInMeta timeout,this will cause regionserver exit,and then
+        // master ServerShutdownHandler will fix daughter & avoid data loss. (See
+        // HBase-4562).
+        updateMetaForDaughterRegions(env);
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR);
+        break;
+      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
+        preSplitRegionAfterPONR(env);
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS);
+        break;
+      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
+        addChildProcedure(createAssignProcedures(env, getRegionReplication(env)));
+        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_POST_OPERATION);
+        break;
+      case SPLIT_TABLE_REGION_POST_OPERATION:
+        postSplitRegion(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      String msg = "Error trying to split region " + parentHRI.getEncodedName() + " in the table "
+          + getTableName() + " (in state=" + state + ")";
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn(msg, e);
+      } else {
+        LOG.error(msg, e);
+        setFailure("master-split-region", e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final SplitTableRegionState state)
+      throws IOException, InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case SPLIT_TABLE_REGION_POST_OPERATION:
+      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
+      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
+      case SPLIT_TABLE_REGION_UPDATE_META:
+        // PONR
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
+        break;
+      case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
+        // Doing nothing, as re-open parent region would clean up daughter region directories.
+        break;
+      case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
+        openParentRegion(env);
+        break;
+      case SPLIT_TABLE_REGION_PRE_OPERATION:
+        postRollBackSplitRegion(env);
+        break;
+      case SPLIT_TABLE_REGION_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for splitting the region "
+        + parentHRI.getEncodedName() + " in table " + getTableName(), e);
+      throw e;
+    }
+  }
+
+  /*
+   * Check whether we are in the state that can be rollback
+   */
+  @Override
+  protected boolean isRollbackSupported(final SplitTableRegionState state) {
+    switch (state) {
+      case SPLIT_TABLE_REGION_POST_OPERATION:
+      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
+      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
+      case SPLIT_TABLE_REGION_UPDATE_META:
+        // It is not safe to rollback if we reach to these states.
+        return false;
+      default:
+        break;
+    }
+    return true;
+  }
+
+  @Override
+  protected SplitTableRegionState getState(final int stateId) {
+    return SplitTableRegionState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final SplitTableRegionState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected SplitTableRegionState getInitialState() {
+    return SplitTableRegionState.SPLIT_TABLE_REGION_PREPARE;
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
+        MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
+        .setParentRegionInfo(HRegionInfo.convert(parentHRI))
+        .addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
+        .addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
+    splitTableRegionMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
+        MasterProcedureProtos.SplitTableRegionStateData.parseDelimitedFrom(stream);
+    setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
+    parentHRI = HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo());
+    assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
+    daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(0));
+    daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(1));
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append("(table=");
+    sb.append(getTableName());
+    sb.append(" parent=");
+    sb.append(parentHRI.getShortNameToLog());
+    sb.append(" daughterA=");
+    sb.append(daughter_1_HRI.getShortNameToLog());
+    sb.append(" daughterB=");
+    sb.append(daughter_2_HRI.getShortNameToLog());
+    sb.append(")");
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
+
+    if (env.getProcedureScheduler().waitRegions(this, getTableName(), parentHRI)) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), parentHRI);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return parentHRI.getTable();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.SPLIT;
+  }
+
+  private byte[] getSplitRow() {
+    return daughter_2_HRI.getStartKey();
+  }
+
+  /**
+   * Prepare to Split region.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  @VisibleForTesting
+  public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
+    // Check whether the region is splittable
+    RegionStateNode node = env.getAssignmentManager().getRegionStates().getRegionNode(parentHRI);
+    if (node != null) {
+      parentHRI = node.getRegionInfo();
+
+      // expected parent to be online or closed
+      if (!node.isInState(State.OPEN, State.CLOSED)) {
+        setFailure("master-split-region",
+          new IOException("Split region " + parentHRI + " failed due to region is not splittable"));
+        return false;
+      }
+
+      // lookup the parent HRI state from the AM, which has the latest updated info.
+      if (parentHRI.isSplit() || parentHRI.isOffline()) {
+        setFailure("master-split-region",
+          new IOException("Split region " + parentHRI + " failed due to region is not splittable"));
+        return false;
+      }
+    }
+
+    // since we have the lock and the master is coordinating the operation
+    // we are always able to split the region
+    if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
+      LOG.warn("split switch is off! skip split of " + parentHRI);
+      setFailure("master-split-region",
+          new IOException("Split region " + parentHRI + " failed due to split switch off"));
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Action before splitting region in a table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preSplitRegion(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preSplitRegionAction(getTableName(), getSplitRow(), getUser());
+    }
+  }
+
+  /**
+   * Action after rollback a split table region action.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void postRollBackSplitRegion(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postRollBackSplitRegionAction(getUser());
+    }
+  }
+
+  /**
+   * Rollback close parent region
+   * @param env MasterProcedureEnv
+   **/
+  private void openParentRegion(final MasterProcedureEnv env) throws IOException {
+    // Check whether the region is closed; if so, open it in the same server
+    final int regionReplication = getRegionReplication(env);
+    final ServerName serverName = getParentRegionServerName(env);
+
+    final AssignProcedure[] procs = new AssignProcedure[regionReplication];
+    for (int i = 0; i < regionReplication; ++i) {
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(parentHRI, i);
+      procs[i] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
+    }
+    env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
+  }
+
+  /**
+   * Create daughter regions
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  @VisibleForTesting
+  public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), parentHRI.getTable());
+    final FileSystem fs = mfs.getFileSystem();
+    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+      env.getMasterConfiguration(), fs, tabledir, parentHRI, false);
+    regionFs.createSplitsDir();
+
+    Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
+
+    assertReferenceFileCount(fs, expectedReferences.getFirst(),
+      regionFs.getSplitsDir(daughter_1_HRI));
+    //Move the files from the temporary .splits to the final /table/region directory
+    regionFs.commitDaughterRegion(daughter_1_HRI);
+    assertReferenceFileCount(fs, expectedReferences.getFirst(),
+      new Path(tabledir, daughter_1_HRI.getEncodedName()));
+
+    assertReferenceFileCount(fs, expectedReferences.getSecond(),
+      regionFs.getSplitsDir(daughter_2_HRI));
+    regionFs.commitDaughterRegion(daughter_2_HRI);
+    assertReferenceFileCount(fs, expectedReferences.getSecond(),
+      new Path(tabledir, daughter_2_HRI.getEncodedName()));
+  }
+
+  /**
+   * Create Split directory
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private Pair<Integer, Integer> splitStoreFiles(
+      final MasterProcedureEnv env,
+      final HRegionFileSystem regionFs) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Configuration conf = env.getMasterConfiguration();
+
+    // The following code sets up a thread pool executor with as many slots as
+    // there's files to split. It then fires up everything, waits for
+    // completion and finally checks for any exception
+    //
+    // Note: splitStoreFiles creates daughter region dirs under the parent splits dir
+    // Nothing to unroll here if failure -- re-run createSplitsDir will
+    // clean this up.
+    int nbFiles = 0;
+    for (String family: regionFs.getFamilies()) {
+      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
+      if (storeFiles != null) {
+        nbFiles += storeFiles.size();
+      }
+    }
+    if (nbFiles == 0) {
+      // no file needs to be splitted.
+      return new Pair<Integer, Integer>(0,0);
+    }
+    // Max #threads is the smaller of the number of storefiles or the default max determined above.
+    int maxThreads = Math.min(
+      conf.getInt(HConstants.REGION_SPLIT_THREADS_MAX,
+        conf.getInt(HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT)),
+      nbFiles);
+    LOG.info("Preparing to split " + nbFiles + " storefiles for region " + parentHRI +
+            " using " + maxThreads + " threads");
+    final ExecutorService threadPool = Executors.newFixedThreadPool(
+      maxThreads, Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
+    final List<Future<Pair<Path,Path>>> futures = new ArrayList<Future<Pair<Path,Path>>>(nbFiles);
+
+    // Split each store file.
+    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
+    for (String family: regionFs.getFamilies()) {
+      final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
+      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
+      if (storeFiles != null && storeFiles.size() > 0) {
+        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
+        for (StoreFileInfo storeFileInfo: storeFiles) {
+          StoreFileSplitter sfs = new StoreFileSplitter(
+            regionFs,
+            family.getBytes(),
+            new StoreFile(
+              mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()));
+          futures.add(threadPool.submit(sfs));
+        }
+      }
+    }
+    // Shutdown the pool
+    threadPool.shutdown();
+
+    // Wait for all the tasks to finish
+    long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 30000);
+    try {
+      boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS);
+      if (stillRunning) {
+        threadPool.shutdownNow();
+        // wait for the thread to shutdown completely.
+        while (!threadPool.isTerminated()) {
+          Thread.sleep(50);
+        }
+        throw new IOException("Took too long to split the" +
+            " files and create the references, aborting split");
+      }
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+    }
+
+    int daughterA = 0;
+    int daughterB = 0;
+    // Look for any exception
+    for (Future<Pair<Path, Path>> future : futures) {
+      try {
+        Pair<Path, Path> p = future.get();
+        daughterA += p.getFirst() != null ? 1 : 0;
+        daughterB += p.getSecond() != null ? 1 : 0;
+      } catch (InterruptedException e) {
+        throw (InterruptedIOException) new InterruptedIOException().initCause(e);
+      } catch (ExecutionException e) {
+        throw new IOException(e);
+      }
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Split storefiles for region " + parentHRI + " Daughter A: " + daughterA
+          + " storefiles, Daughter B: " + daughterB + " storefiles.");
+    }
+    return new Pair<Integer, Integer>(daughterA, daughterB);
+  }
+
+  private void assertReferenceFileCount(final FileSystem fs, final int expectedReferenceFileCount,
+      final Path dir) throws IOException {
+    if (expectedReferenceFileCount != 0 &&
+        expectedReferenceFileCount != FSUtils.getRegionReferenceFileCount(fs, dir)) {
+      throw new IOException("Failing split. Expected reference file count isn't equal.");
+    }
+  }
+
+  private Pair<Path, Path> splitStoreFile(final HRegionFileSystem regionFs,
+      final byte[] family, final StoreFile sf) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Splitting started for store file: " + sf.getPath() + " for region: " + parentHRI);
+    }
+
+    final byte[] splitRow = getSplitRow();
+    final String familyName = Bytes.toString(family);
+    final Path path_first =
+        regionFs.splitStoreFile(this.daughter_1_HRI, familyName, sf, splitRow, false, null);
+    final Path path_second =
+        regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, splitRow, true, null);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Splitting complete for store file: " + sf.getPath() + " for region: " + parentHRI);
+    }
+    return new Pair<Path,Path>(path_first, path_second);
+  }
+
+  /**
+   * Utility class used to do the file splitting / reference writing
+   * in parallel instead of sequentially.
+   */
+  private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
+    private final HRegionFileSystem regionFs;
+    private final byte[] family;
+    private final StoreFile sf;
+
+    /**
+     * Constructor that takes what it needs to split
+     * @param regionFs the file system
+     * @param family Family that contains the store file
+     * @param sf which file
+     */
+    public StoreFileSplitter(final HRegionFileSystem regionFs, final byte[] family,
+        final StoreFile sf) {
+      this.regionFs = regionFs;
+      this.sf = sf;
+      this.family = family;
+    }
+
+    public Pair<Path,Path> call() throws IOException {
+      return splitStoreFile(regionFs, family, sf);
+    }
+  }
+
+  /**
+   * Post split region actions before the Point-of-No-Return step
+   * @param env MasterProcedureEnv
+   **/
+  private void preSplitRegionBeforePONR(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final List<Mutation> metaEntries = new ArrayList<Mutation>();
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      if (cpHost.preSplitBeforePONRAction(getSplitRow(), metaEntries, getUser())) {
+        throw new IOException("Coprocessor bypassing region " +
+            parentHRI.getRegionNameAsString() + " split.");
+      }
+      try {
+        for (Mutation p : metaEntries) {
+          HRegionInfo.parseRegionName(p.getRow());
+        }
+      } catch (IOException e) {
+        LOG.error("Row key of mutation from coprocessor is not parsable as region name."
+            + "Mutations from coprocessor should only for hbase:meta table.");
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Add daughter regions to META
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
+    env.getAssignmentManager().markRegionAsSplitted(parentHRI, getParentRegionServerName(env),
+      daughter_1_HRI, daughter_2_HRI);
+  }
+
+  /**
+   * Pre split region actions after the Point-of-No-Return step
+   * @param env MasterProcedureEnv
+   **/
+  private void preSplitRegionAfterPONR(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preSplitAfterPONRAction(getUser());
+    }
+  }
+
+  /**
+   * Post split region actions
+   * @param env MasterProcedureEnv
+   **/
+  private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postCompletedSplitRegionAction(daughter_1_HRI, daughter_2_HRI, getUser());
+    }
+  }
+
+  private ServerName getParentRegionServerName(final MasterProcedureEnv env) {
+    return env.getMasterServices().getAssignmentManager()
+      .getRegionStates().getRegionServerOfRegion(parentHRI);
+  }
+
+  private UnassignProcedure[] createUnassignProcedures(final MasterProcedureEnv env,
+      final int regionReplication) {
+    final UnassignProcedure[] procs = new UnassignProcedure[regionReplication];
+    for (int i = 0; i < procs.length; ++i) {
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(parentHRI, i);
+      procs[i] = env.getAssignmentManager().createUnassignProcedure(hri, null, true);
+    }
+    return procs;
+  }
+
+  private AssignProcedure[] createAssignProcedures(final MasterProcedureEnv env,
+      final int regionReplication) {
+    final ServerName targetServer = getParentRegionServerName(env);
+    final AssignProcedure[] procs = new AssignProcedure[regionReplication * 2];
+    int procsIdx = 0;
+    for (int i = 0; i < regionReplication; ++i) {
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_HRI, i);
+      procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
+    }
+    for (int i = 0; i < regionReplication; ++i) {
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_HRI, i);
+      procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
+    }
+    return procs;
+  }
+
+  private int getRegionReplication(final MasterProcedureEnv env) throws IOException {
+    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
+    return htd.getRegionReplication();
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
new file mode 100644
index 0000000..02c7996
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
@@ -0,0 +1,216 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UnassignRegionStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+
+
+/**
+ * Procedure that describe the unassignment of a single region.
+ * There can only be one RegionTransitionProcedure per region running at the time,
+ * since each procedure takes a lock on the region.
+ *
+ * <p>The Unassign starts by placing a "close region" request in the Remote Dispatcher
+ * queue, and the procedure will then go into a "waiting state".
+ * The Remote Dispatcher will batch the various requests for that server and
+ * they will be sent to the RS for execution.
+ * The RS will complete the open operation by calling master.reportRegionStateTransition().
+ * The AM will intercept the transition report, and notify the procedure.
+ * The procedure will finish the unassign by publishing its new state on meta
+ * or it will retry the unassign.
+ */
+@InterfaceAudience.Private
+public class UnassignProcedure extends RegionTransitionProcedure {
+  private static final Log LOG = LogFactory.getLog(UnassignProcedure.class);
+
+  private final AtomicBoolean serverCrashed = new AtomicBoolean(false);
+
+  // TODO: should this be in a reassign procedure?
+  //       ...and keep unassign for 'disable' case?
+  private ServerName destinationServer;
+  private boolean force;
+
+  public UnassignProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  public UnassignProcedure(final HRegionInfo regionInfo,
+      final ServerName destinationServer, final boolean force) {
+    super(regionInfo);
+    this.destinationServer = destinationServer;
+    this.force = force;
+
+    // we don't need REGION_TRANSITION_QUEUE, we jump directly to sending the request
+    setTransitionState(RegionTransitionState.REGION_TRANSITION_DISPATCH);
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.UNASSIGN;
+  }
+
+  @Override
+  protected boolean isRollbackSupported(final RegionTransitionState state) {
+    switch (state) {
+      case REGION_TRANSITION_QUEUE:
+      case REGION_TRANSITION_DISPATCH:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    final UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
+        .setTransitionState(getTransitionState())
+        .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
+    if (destinationServer != null) {
+      state.setDestinationServer(ProtobufUtil.toServerName(destinationServer));
+    }
+    if (force) {
+      state.setForce(true);
+    }
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    final UnassignRegionStateData state = UnassignRegionStateData.parseDelimitedFrom(stream);
+    setTransitionState(state.getTransitionState());
+    setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
+    force = state.getForce();
+    if (state.hasDestinationServer()) {
+      destinationServer = ProtobufUtil.toServerName(state.getDestinationServer());
+    }
+  }
+
+  @Override
+  protected boolean startTransition(final MasterProcedureEnv env, final RegionStateNode regionNode) {
+    // nothing to do here. we skip the step in the constructor
+    // by jumping to REGION_TRANSITION_DISPATCH
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean updateTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
+        throws IOException {
+    // if the region is already closed or offline we can't do much...
+    if (regionNode.isInState(State.CLOSED, State.OFFLINE)) {
+      LOG.info("Not unassigned " + regionNode);
+      return false;
+    }
+
+    // if the server is down, mark the operation as complete
+    if (serverCrashed.get() || !isServerOnline(env, regionNode)) {
+      LOG.info("Server already down: " + regionNode);
+      return false;
+    }
+
+    // if we haven't started the operation yet, we can abort
+    if (aborted.get() && regionNode.isInState(State.OPEN)) {
+      setAbortFailure(getClass().getSimpleName(), "abort requested");
+      return false;
+    }
+
+    // Mark the region as closing
+    env.getAssignmentManager().markRegionAsClosing(regionNode);
+
+    // Add the close region operation the the server dispatch queue.
+    // The pending close will be dispatched to the server together with the other
+    // pending operation for that server.
+    addToRemoteDispatcher(env, regionNode.getRegionLocation());
+    return true;
+  }
+
+  @Override
+  protected void completeTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
+      throws IOException {
+    env.getAssignmentManager().markRegionAsClosed(regionNode);
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(final MasterProcedureEnv env, final ServerName serverName) {
+    assert serverName.equals(getRegionState(env).getRegionLocation());
+    return new RegionCloseOperation(this, getRegionInfo(), destinationServer);
+  }
+
+  @Override
+  protected void reportTransition(final MasterProcedureEnv env, final RegionStateNode regionNode,
+      final TransitionCode code, final long seqId) throws UnexpectedStateException {
+    switch (code) {
+      case CLOSED:
+        setTransitionState(RegionTransitionState.REGION_TRANSITION_FINISH);
+        break;
+      default:
+        throw new UnexpectedStateException(String.format(
+          "reported unexpected transition state=%s for region=%s server=%s, expected CLOSED.",
+          code, regionNode.getRegionInfo(), regionNode.getRegionLocation()));
+    }
+  }
+
+  @Override
+  protected void remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
+      final IOException exception) {
+    if (exception instanceof RegionServerAbortedException ||
+        exception instanceof RegionServerStoppedException ||
+        exception instanceof ServerNotRunningYetException) {
+      // TODO
+      // RS is aborting, we cannot offline the region since the region may need to do WAL
+      // recovery. Until we see  the RS expiration, we should retry.
+      serverCrashed.set(true);
+    } else if (exception instanceof NotServingRegionException) {
+      // TODO
+      serverCrashed.set(true);
+    } else {
+      // TODO: kill the server in case we get an exception we are not able to handle
+      LOG.warn("killing server because we are not able to close the region=" + regionNode +
+        " exception=" + exception);
+      env.getMasterServices().getServerManager().expireServer(regionNode.getRegionLocation());
+      serverCrashed.set(true);
+    }
+    // ...TODO
+    setTransitionState(RegionTransitionState.REGION_TRANSITION_DISPATCH);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 0f1b1a2..8d45494 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -63,7 +63,7 @@ import com.google.common.collect.Sets;
 
 /**
  * The base class for load balancers. It provides the the functions used to by
- * {@link org.apache.hadoop.hbase.master.AssignmentManager} to assign regions
+ * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager} to assign regions
  * in the edge cases. It doesn't provide an implementation of the
  * actual balancing algorithm.
  *

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
index f7e166d..907e745 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/RegionLocationFinder.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
@@ -39,9 +38,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
@@ -149,19 +147,15 @@ class RegionLocationFinder {
     if (services == null) {
       return false;
     }
-    AssignmentManager am = services.getAssignmentManager();
 
+    final AssignmentManager am = services.getAssignmentManager();
     if (am == null) {
       return false;
     }
-    RegionStates regionStates = am.getRegionStates();
-    if (regionStates == null) {
-      return false;
-    }
 
-    Set<HRegionInfo> regions = regionStates.getRegionAssignments().keySet();
+    // TODO: Should this refresh all the regions or only the ones assigned?
     boolean includesUserTables = false;
-    for (final HRegionInfo hri : regions) {
+    for (final HRegionInfo hri : am.getAssignedRegions()) {
       cache.refresh(hri);
       includesUserTables = includesUserTables || !hri.isSystemTable();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
index 7e8d696..818156d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/SimpleLoadBalancer.java
@@ -20,28 +20,27 @@ package org.apache.hadoop.hbase.master.balancer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Random;
 import java.util.TreeMap;
-import java.util.Comparator;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.util.Pair;
 
 import com.google.common.collect.MinMaxPriorityQueue;
-import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * Makes decisions about the placement and movement of Regions across
@@ -54,7 +53,7 @@ import org.apache.hadoop.hbase.util.Pair;
  * locations for all Regions in a cluster.
  *
  * <p>This classes produces plans for the
- * {@link org.apache.hadoop.hbase.master.AssignmentManager} to execute.
+ * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager} to execute.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class SimpleLoadBalancer extends BaseLoadBalancer {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index 59ea067..01058d8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -30,7 +30,7 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@@ -156,23 +157,16 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
   @Override
   public synchronized void setConf(Configuration conf) {
     super.setConf(conf);
-    LOG.info("loading config");
-
     maxSteps = conf.getInt(MAX_STEPS_KEY, maxSteps);
-
     stepsPerRegion = conf.getInt(STEPS_PER_REGION_KEY, stepsPerRegion);
     maxRunningTime = conf.getLong(MAX_RUNNING_TIME_KEY, maxRunningTime);
-
     numRegionLoadsToRemember = conf.getInt(KEEP_REGION_LOADS, numRegionLoadsToRemember);
     isByTable = conf.getBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, isByTable);
-
     minCostNeedBalance = conf.getFloat(MIN_COST_NEED_BALANCE_KEY, minCostNeedBalance);
-
     if (localityCandidateGenerator == null) {
       localityCandidateGenerator = new LocalityBasedCandidateGenerator(services);
     }
     localityCost = new LocalityCostFunction(conf, services);
-
     if (candidateGenerators == null) {
       candidateGenerators = new CandidateGenerator[] {
           new RandomCandidateGenerator(),
@@ -181,17 +175,14 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
           new RegionReplicaRackCandidateGenerator(),
       };
     }
-
     regionLoadFunctions = new CostFromRegionLoadFunction[] {
       new ReadRequestCostFunction(conf),
       new WriteRequestCostFunction(conf),
       new MemstoreSizeCostFunction(conf),
       new StoreFileCostFunction(conf)
     };
-
     regionReplicaHostCostFunction = new RegionReplicaHostCostFunction(conf);
     regionReplicaRackCostFunction = new RegionReplicaRackCostFunction(conf);
-
     costFunctions = new CostFunction[]{
       new RegionCountSkewCostFunction(conf),
       new PrimaryRegionCountSkewCostFunction(conf),
@@ -205,10 +196,10 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
       regionLoadFunctions[2],
       regionLoadFunctions[3],
     };
-
     curFunctionCosts= new Double[costFunctions.length];
     tempFunctionCosts= new Double[costFunctions.length];
-
+    LOG.info("Loaded config; maxSteps=" + maxSteps + ", stepsPerRegion=" + stepsPerRegion +
+        ", maxRunningTime=" + maxRunningTime + ", isByTable=" + isByTable + ", etc.");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index 8e490eb..7f68654 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -43,11 +43,15 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * Procedure to allow clients and external admin tools to take locks on table/namespace/regions.
- * This procedure when scheduled, acquires specified locks, suspends itself and waits for :
- * - call to unlock: if lock request came from the process itself, say master chore.
- * - Timeout : if lock request came from RPC. On timeout, evaluates if it should continue holding
- * the lock or not based on last heartbeat timestamp.
+ * Procedure to allow blessed clients and external admin tools to take our internal Schema locks
+ * used by the procedure framework isolating procedures doing creates/deletes etc. on
+ * table/namespace/regions.
+ * This procedure when scheduled, acquires specified locks, suspends itself and waits for:
+ * <ul>
+ * <li>Call to unlock: if lock request came from the process itself, say master chore.</li>
+ * <li>Timeout : if lock request came from RPC. On timeout, evaluates if it should continue holding
+ * the lock or not based on last heartbeat timestamp.</li>
+ * </ul>
  */
 @InterfaceAudience.Private
 public final class LockProcedure extends Procedure<MasterProcedureEnv>
@@ -191,7 +195,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   public void updateHeartBeat() {
     lastHeartBeat.set(System.currentTimeMillis());
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Update heartbeat. Proc: " + toString());
+      LOG.debug("Heartbeat " + toString());
     }
   }
 
@@ -202,8 +206,10 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
    */
   protected boolean setTimeoutFailure(final MasterProcedureEnv env) {
     synchronized (event) {
-      if (!event.isReady()) {  // maybe unlock() awakened the event.
+      if (LOG.isTraceEnabled()) LOG.info("Timeout failure " + this.event);
+      if (!event.isReady()) {  // Maybe unlock() awakened the event.
         setState(ProcedureProtos.ProcedureState.RUNNABLE);
+        if (LOG.isTraceEnabled()) LOG.info("Calling wake on " + this.event);
         env.getProcedureScheduler().wakeEvent(event);
       }
     }
@@ -234,7 +240,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     }
     if (unlock.get() || hasHeartbeatExpired()) {
       locked.set(false);
-      LOG.debug((unlock.get() ? "UNLOCKED - " : "TIMED OUT - ") + toString());
+      LOG.debug((unlock.get()? "UNLOCKED " : "TIMED OUT ") + toString());
       return null;
     }
     synchronized (event) {
@@ -302,7 +308,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     hasLock = ret;
     if (ret) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("LOCKED - " + toString());
+        LOG.debug("LOCKED " + toString());
       }
       lastHeartBeat.set(System.currentTimeMillis());
       return LockState.LOCK_ACQUIRED;
@@ -352,7 +358,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
     } else if (tableName != null) {
       return setupTableLock();
     } else {
-      LOG.error("Unknown level specified in proc - " + toString());
+      LOG.error("Unknown level specified in " + toString());
       throw new IllegalArgumentException("no namespace/table/region provided");
     }
   }
@@ -364,10 +370,10 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
         this.opType = TableOperationType.EDIT;
         return new NamespaceExclusiveLock();
       case SHARED:
-        LOG.error("Shared lock on namespace not supported. Proc - " + toString());
+        LOG.error("Shared lock on namespace not supported for " + toString());
         throw new IllegalArgumentException("Shared lock on namespace not supported");
       default:
-        LOG.error("Unexpected lock type in proc - " + toString());
+        LOG.error("Unexpected lock type " + toString());
         throw new IllegalArgumentException("Wrong lock type: " + type.toString());
     }
   }
@@ -381,7 +387,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
         this.opType = TableOperationType.READ;
         return new TableSharedLock();
       default:
-        LOG.error("Unexpected lock type in proc - " + toString());
+        LOG.error("Unexpected lock type " + toString());
         throw new IllegalArgumentException("Wrong lock type:" + type.toString());
     }
   }
@@ -393,7 +399,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
         this.opType = TableOperationType.REGION_EDIT;
         return new RegionExclusiveLock();
       default:
-        LOG.error("Only exclusive lock supported on regions. Proc - " + toString());
+        LOG.error("Only exclusive lock supported on regions for " + toString());
         throw new IllegalArgumentException("Only exclusive lock supported on regions.");
     }
   }
@@ -477,4 +483,4 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
       env.getProcedureScheduler().wakeRegions(LockProcedure.this, tableName, regionInfos);
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
index e957f9d..312d7fe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -111,4 +111,4 @@ public abstract class AbstractStateMachineTableProcedure<TState>
       throw new TableNotFoundException(getTableName());
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
index 7bb2887..34c1853 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -100,7 +99,10 @@ public class AddColumnFamilyProcedure
         setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        reOpenAllRegionsIfTableIsOnline(env);
+        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
+          addChildProcedure(env.getAssignmentManager()
+            .createReopenProcedures(getRegionInfoList(env)));
+        }
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -285,7 +287,8 @@ public class AddColumnFamilyProcedure
       env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
       // Make sure regions are opened after table descriptor is updated.
-      reOpenAllRegionsIfTableIsOnline(env);
+      //reOpenAllRegionsIfTableIsOnline(env);
+      // TODO: NUKE ROLLBACK!!!!
     }
   }
 
@@ -302,25 +305,6 @@ public class AddColumnFamilyProcedure
   }
 
   /**
-   * Last action from the procedure - executed when online schema change is supported.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
-    // This operation only run when the table is enabled.
-    if (!env.getMasterServices().getTableStateManager()
-        .isTableState(getTableName(), TableState.State.ENABLED)) {
-      return;
-    }
-
-    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), getRegionInfoList(env))) {
-      LOG.info("Completed add column family operation on table " + getTableName());
-    } else {
-      LOG.warn("Error on reopening the regions on table " + getTableName());
-    }
-  }
-
-  /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
    * @return traceEnabled
@@ -362,7 +346,8 @@ public class AddColumnFamilyProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+      regionInfoList = env.getAssignmentManager().getRegionStates()
+          .getRegionsOfTable(getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index 347d01d..fc2a4ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -133,10 +133,12 @@ public class CloneSnapshotProcedure
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_ASSIGN_REGIONS);
           break;
         case CLONE_SNAPSHOT_ASSIGN_REGIONS:
-          CreateTableProcedure.assignRegions(env, getTableName(), newRegions);
+          CreateTableProcedure.setEnablingState(env, getTableName());
+          addChildProcedure(env.getAssignmentManager().createAssignProcedures(newRegions));
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_UPDATE_DESC_CACHE);
           break;
         case CLONE_SNAPSHOT_UPDATE_DESC_CACHE:
+          CreateTableProcedure.setEnabledState(env, getTableName());
           CreateTableProcedure.updateTableDescCache(env, getTableName());
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_POST_OPERATION);
           break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 2c39c09..7d65126 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -62,7 +62,6 @@ public class CreateNamespaceProcedure
     if (isTraceEnabled()) {
       LOG.trace(this + " execute state=" + state);
     }
-
     try {
       switch (state) {
       case CREATE_NAMESPACE_PREPARE:

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index ced7abc..c3900dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -107,10 +106,12 @@ public class CreateTableProcedure
           setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
           break;
         case CREATE_TABLE_ASSIGN_REGIONS:
-          assignRegions(env, getTableName(), newRegions);
+          setEnablingState(env, getTableName());
+          addChildProcedure(env.getAssignmentManager().createAssignProcedures(newRegions));
           setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
           break;
         case CREATE_TABLE_UPDATE_DESC_CACHE:
+          setEnabledState(env, getTableName());
           updateTableDescCache(env, getTableName());
           setNextState(CreateTableState.CREATE_TABLE_POST_OPERATION);
           break;
@@ -333,21 +334,21 @@ public class CreateTableProcedure
   protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
       final HTableDescriptor hTableDescriptor,
       final List<HRegionInfo> regions) throws IOException {
-    if (regions != null && regions.size() > 0) {
-      ProcedureSyncWait.waitMetaRegions(env);
+    assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
 
-      // Add regions to META
-      addRegionsToMeta(env, hTableDescriptor, regions);
-      // Add replicas if needed
-      List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
+    ProcedureSyncWait.waitMetaRegions(env);
 
-      // Setup replication for region replicas if needed
-      if (hTableDescriptor.getRegionReplication() > 1) {
-        ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
-      }
-      return newRegions;
+    // Add replicas if needed
+    List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
+
+    // Add regions to META
+    addRegionsToMeta(env, hTableDescriptor, newRegions);
+
+    // Setup replication for region replicas if needed
+    if (hTableDescriptor.getRegionReplication() > 1) {
+      ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
     }
-    return regions;
+    return newRegions;
   }
 
   /**
@@ -374,18 +375,16 @@ public class CreateTableProcedure
     return hRegionInfos;
   }
 
-  protected static void assignRegions(final MasterProcedureEnv env,
-      final TableName tableName, final List<HRegionInfo> regions) throws IOException {
-    ProcedureSyncWait.waitRegionServers(env);
 
+  protected static void setEnablingState(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
     // Mark the table as Enabling
     env.getMasterServices().getTableStateManager()
       .setTableState(tableName, TableState.State.ENABLING);
+  }
 
-    // Trigger immediate assignment of the regions in round-robin fashion
-    final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
-    ModifyRegionUtils.assignRegions(assignmentManager, regions);
-
+  protected static void setEnabledState(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
     // Enable table
     env.getMasterServices().getTableStateManager()
       .setTableState(tableName, TableState.State.ENABLED);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
index 096172a..78bd715 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -106,7 +105,10 @@ public class DeleteColumnFamilyProcedure
         setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        reOpenAllRegionsIfTableIsOnline(env);
+        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
+          addChildProcedure(env.getAssignmentManager()
+            .createReopenProcedures(getRegionInfoList(env)));
+        }
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -292,7 +294,8 @@ public class DeleteColumnFamilyProcedure
     env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    reOpenAllRegionsIfTableIsOnline(env);
+    //reOpenAllRegionsIfTableIsOnline(env);
+    // TODO: NUKE ROLLBACK!!!!
   }
 
   /**
@@ -316,25 +319,6 @@ public class DeleteColumnFamilyProcedure
   }
 
   /**
-   * Last action from the procedure - executed when online schema change is supported.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
-    // This operation only run when the table is enabled.
-    if (!env.getMasterServices().getTableStateManager()
-        .isTableState(getTableName(), TableState.State.ENABLED)) {
-      return;
-    }
-
-    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), getRegionInfoList(env))) {
-      LOG.info("Completed delete column family operation on table " + getTableName());
-    } else {
-      LOG.warn("Error on reopening the regions on table " + getTableName());
-    }
-  }
-
-  /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
    * @return traceEnabled
@@ -376,7 +360,8 @@ public class DeleteColumnFamilyProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+      regionInfoList = env.getAssignmentManager().getRegionStates()
+          .getRegionsOfTable(getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 9d0a283..5770067 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -98,7 +97,7 @@ public class DeleteTableProcedure
 
           // TODO: Move out... in the acquireLock()
           LOG.debug("waiting for '" + getTableName() + "' regions in transition");
-          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+          regions = env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName());
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 
@@ -341,8 +340,7 @@ public class DeleteTableProcedure
       final TableName tableName) throws IOException {
     Connection connection = env.getMasterServices().getConnection();
     Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
-    try (Table metaTable =
-        connection.getTable(TableName.META_TABLE_NAME)) {
+    try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
       List<Delete> deletes = new ArrayList<>();
       try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
         for (Result result : resScanner) {
@@ -376,11 +374,9 @@ public class DeleteTableProcedure
 
   protected static void deleteAssignmentState(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
-    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
-
     // Clean up regions of the table in RegionStates.
     LOG.debug("Removing '" + tableName + "' from region states.");
-    am.getRegionStates().tableDeleted(tableName);
+    env.getMasterServices().getAssignmentManager().deleteTable(tableName);
 
     // If entry for this table states, remove it.
     LOG.debug("Marking '" + tableName + "' as deleted.");


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
deleted file mode 100644
index 69ebd97..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ /dev/null
@@ -1,3053 +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;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateException;
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.RegionStateListener;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.MasterSwitchType;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.executor.EventHandler;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.executor.ExecutorService;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-import org.apache.hadoop.hbase.ipc.FailedServerException;
-import org.apache.hadoop.hbase.ipc.RpcClient;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.quotas.QuotaExceededException;
-import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
-import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.KeyLocker;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hadoop.hbase.util.RetryCounter;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * Manages and performs region assignment.
- * Related communications with regionserver are all done over RPC.
- */
-@InterfaceAudience.Private
-public class AssignmentManager {
-  private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
-
-  protected final MasterServices server;
-
-  private ServerManager serverManager;
-
-  private boolean shouldAssignRegionsWithFavoredNodes;
-
-  private LoadBalancer balancer;
-
-  private final MetricsAssignmentManager metricsAssignmentManager;
-
-  private AtomicInteger numRegionsOpened = new AtomicInteger(0);
-
-  final private KeyLocker<String> locker = new KeyLocker<>();
-
-  Set<HRegionInfo> replicasToClose = Collections.synchronizedSet(new HashSet<HRegionInfo>());
-
-  /**
-   * Map of regions to reopen after the schema of a table is changed. Key -
-   * encoded region name, value - HRegionInfo
-   */
-  private final Map <String, HRegionInfo> regionsToReopen;
-
-  /*
-   * Maximum times we recurse an assignment/unassignment.
-   * See below in {@link #assign()} and {@link #unassign()}.
-   */
-  private final int maximumAttempts;
-
-  /**
-   * The sleep time for which the assignment will wait before retrying in case of
-   * hbase:meta assignment failure due to lack of availability of region plan or bad region plan
-   */
-  private final long sleepTimeBeforeRetryingMetaAssignment;
-
-  /** Plans for region movement. Key is the encoded version of a region name*/
-  // TODO: When do plans get cleaned out?  Ever? In server open and in server
-  // shutdown processing -- St.Ack
-  // All access to this Map must be synchronized.
-  final NavigableMap<String, RegionPlan> regionPlans = new TreeMap<>();
-
-  private final TableStateManager tableStateManager;
-
-  private final ExecutorService executorService;
-
-  private java.util.concurrent.ExecutorService threadPoolExecutorService;
-  private ScheduledThreadPoolExecutor scheduledThreadPoolExecutor;
-
-  private final RegionStates regionStates;
-
-  // The threshold to use bulk assigning. Using bulk assignment
-  // only if assigning at least this many regions to at least this
-  // many servers. If assigning fewer regions to fewer servers,
-  // bulk assigning may be not as efficient.
-  private final int bulkAssignThresholdRegions;
-  private final int bulkAssignThresholdServers;
-  private final int bulkPerRegionOpenTimeGuesstimate;
-
-  // Should bulk assignment wait till all regions are assigned,
-  // or it is timed out?  This is useful to measure bulk assignment
-  // performance, but not needed in most use cases.
-  private final boolean bulkAssignWaitTillAllAssigned;
-
-  /**
-   * Indicator that AssignmentManager has recovered the region states so
-   * that ServerShutdownHandler can be fully enabled and re-assign regions
-   * of dead servers. So that when re-assignment happens, AssignmentManager
-   * has proper region states.
-   *
-   * Protected to ease testing.
-   */
-  protected final AtomicBoolean failoverCleanupDone = new AtomicBoolean(false);
-
-  /**
-   * A map to track the count a region fails to open in a row.
-   * So that we don't try to open a region forever if the failure is
-   * unrecoverable.  We don't put this information in region states
-   * because we don't expect this to happen frequently; we don't
-   * want to copy this information over during each state transition either.
-   */
-  private final ConcurrentHashMap<String, AtomicInteger> failedOpenTracker = new ConcurrentHashMap<>();
-
-  // In case not using ZK for region assignment, region states
-  // are persisted in meta with a state store
-  private final RegionStateStore regionStateStore;
-
-  /**
-   * For testing only!  Set to true to skip handling of split.
-   */
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
-  public static boolean TEST_SKIP_SPLIT_HANDLING = false;
-
-  /** Listeners that are called on assignment events. */
-  private List<AssignmentListener> listeners = new CopyOnWriteArrayList<>();
-
-  private RegionStateListener regionStateListener;
-
-  private RetryCounter.BackoffPolicy backoffPolicy;
-  private RetryCounter.RetryConfig retryConfig;
-  /**
-   * Constructs a new assignment manager.
-   *
-   * @param server instance of HMaster this AM running inside
-   * @param serverManager serverManager for associated HMaster
-   * @param balancer implementation of {@link LoadBalancer}
-   * @param service Executor service
-   * @param metricsMaster metrics manager
-   * @throws IOException
-   */
-  public AssignmentManager(MasterServices server, ServerManager serverManager,
-      final LoadBalancer balancer, final ExecutorService service, MetricsMaster metricsMaster,
-      final TableStateManager tableStateManager)
-          throws IOException {
-    this.server = server;
-    this.serverManager = serverManager;
-    this.executorService = service;
-    this.regionStateStore = new RegionStateStore(server);
-    this.regionsToReopen = Collections.synchronizedMap
-                           (new HashMap<String, HRegionInfo> ());
-    Configuration conf = server.getConfiguration();
-
-    this.tableStateManager = tableStateManager;
-
-    // This is the max attempts, not retries, so it should be at least 1.
-    this.maximumAttempts = Math.max(1,
-      this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
-    this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong(
-        "hbase.meta.assignment.retry.sleeptime", 1000l);
-    this.balancer = balancer;
-    // Only read favored nodes if using the favored nodes load balancer.
-    this.shouldAssignRegionsWithFavoredNodes = this.balancer instanceof FavoredNodesPromoter;
-    int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
-
-    this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
-        maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
-
-    this.scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor(1,
-        Threads.newDaemonThreadFactory("AM.Scheduler"));
-
-    this.regionStates = new RegionStates(
-      server, tableStateManager, serverManager, regionStateStore);
-
-    this.bulkAssignWaitTillAllAssigned =
-      conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
-    this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
-    this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
-    this.bulkPerRegionOpenTimeGuesstimate =
-      conf.getInt("hbase.bulk.assignment.perregion.open.time", 10000);
-
-    this.metricsAssignmentManager = new MetricsAssignmentManager();
-
-    // Configurations for retrying opening a region on receiving a FAILED_OPEN
-    this.retryConfig = new RetryCounter.RetryConfig();
-    this.retryConfig.setSleepInterval(conf.getLong("hbase.assignment.retry.sleep.initial", 0l));
-    // Set the max time limit to the initial sleep interval so we use a constant time sleep strategy
-    // if the user does not set a max sleep limit
-    this.retryConfig.setMaxSleepTime(conf.getLong("hbase.assignment.retry.sleep.max",
-        retryConfig.getSleepInterval()));
-    this.backoffPolicy = getBackoffPolicy();
-  }
-
-  /**
-   * Returns the backoff policy used for Failed Region Open retries
-   * @return the backoff policy used for Failed Region Open retries
-   */
-  RetryCounter.BackoffPolicy getBackoffPolicy() {
-    return new RetryCounter.ExponentialBackoffPolicyWithLimit();
-  }
-
-  MetricsAssignmentManager getAssignmentManagerMetrics() {
-    return this.metricsAssignmentManager;
-  }
-
-  /**
-   * Add the listener to the notification list.
-   * @param listener The AssignmentListener to register
-   */
-  public void registerListener(final AssignmentListener listener) {
-    this.listeners.add(listener);
-  }
-
-  /**
-   * Remove the listener from the notification list.
-   * @param listener The AssignmentListener to unregister
-   */
-  public boolean unregisterListener(final AssignmentListener listener) {
-    return this.listeners.remove(listener);
-  }
-
-  /**
-   * @return Instance of ZKTableStateManager.
-   */
-  public TableStateManager getTableStateManager() {
-    // These are 'expensive' to make involving trip to zk ensemble so allow
-    // sharing.
-    return this.tableStateManager;
-  }
-
-  /**
-   * This SHOULD not be public. It is public now
-   * because of some unit tests.
-   *
-   * TODO: make it package private and keep RegionStates in the master package
-   */
-  public RegionStates getRegionStates() {
-    return regionStates;
-  }
-
-  /**
-   * Used in some tests to mock up region state in meta
-   */
-  @VisibleForTesting
-  RegionStateStore getRegionStateStore() {
-    return regionStateStore;
-  }
-
-  public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
-    return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
-  }
-
-  /**
-   * Add a regionPlan for the specified region.
-   * @param encodedName
-   * @param plan
-   */
-  public void addPlan(String encodedName, RegionPlan plan) {
-    synchronized (regionPlans) {
-      regionPlans.put(encodedName, plan);
-    }
-  }
-
-  /**
-   * Add a map of region plans.
-   */
-  public void addPlans(Map<String, RegionPlan> plans) {
-    synchronized (regionPlans) {
-      regionPlans.putAll(plans);
-    }
-  }
-
-  /**
-   * Set the list of regions that will be reopened
-   * because of an update in table schema
-   *
-   * @param regions
-   *          list of regions that should be tracked for reopen
-   */
-  public void setRegionsToReopen(List <HRegionInfo> regions) {
-    for(HRegionInfo hri : regions) {
-      regionsToReopen.put(hri.getEncodedName(), hri);
-    }
-  }
-
-  /**
-   * Used by the client to identify if all regions have the schema updates
-   *
-   * @param tableName
-   * @return Pair indicating the status of the alter command
-   * @throws IOException
-   */
-  public Pair<Integer, Integer> getReopenStatus(TableName tableName)
-      throws IOException {
-    List<HRegionInfo> hris;
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
-      hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper());
-    } else {
-      hris = MetaTableAccessor.getTableRegions(server.getConnection(), tableName, true);
-    }
-
-    Integer pending = 0;
-    for (HRegionInfo hri : hris) {
-      String name = hri.getEncodedName();
-      // no lock concurrent access ok: sequential consistency respected.
-      if (regionsToReopen.containsKey(name)
-          || regionStates.isRegionInTransition(name)) {
-        pending++;
-      }
-    }
-    return new Pair<>(pending, hris.size());
-  }
-
-  /**
-   * Used by ServerShutdownHandler to make sure AssignmentManager has completed
-   * the failover cleanup before re-assigning regions of dead servers. So that
-   * when re-assignment happens, AssignmentManager has proper region states.
-   */
-  public boolean isFailoverCleanupDone() {
-    return failoverCleanupDone.get();
-  }
-
-  /**
-   * To avoid racing with AM, external entities may need to lock a region,
-   * for example, when SSH checks what regions to skip re-assigning.
-   */
-  public Lock acquireRegionLock(final String encodedName) {
-    return locker.acquireLock(encodedName);
-  }
-
-  /**
-   * Now, failover cleanup is completed. Notify server manager to
-   * process queued up dead servers processing, if any.
-   */
-  void failoverCleanupDone() {
-    failoverCleanupDone.set(true);
-    serverManager.processQueuedDeadServers();
-  }
-
-  /**
-   * Called on startup.
-   * Figures whether a fresh cluster start of we are joining extant running cluster.
-   * @throws IOException
-   * @throws KeeperException
-   * @throws InterruptedException
-   * @throws CoordinatedStateException
-   */
-  void joinCluster()
-  throws IOException, KeeperException, InterruptedException, CoordinatedStateException {
-    long startTime = System.currentTimeMillis();
-    // Concurrency note: In the below the accesses on regionsInTransition are
-    // outside of a synchronization block where usually all accesses to RIT are
-    // synchronized.  The presumption is that in this case it is safe since this
-    // method is being played by a single thread on startup.
-
-    // TODO: Regions that have a null location and are not in regionsInTransitions
-    // need to be handled.
-
-    // Scan hbase:meta to build list of existing regions, servers, and assignment
-    // Returns servers who have not checked in (assumed dead) that some regions
-    // were assigned to (according to the meta)
-    Set<ServerName> deadServers = rebuildUserRegions();
-
-    // This method will assign all user regions if a clean server startup or
-    // it will reconstruct master state and cleanup any leftovers from previous master process.
-    boolean failover = processDeadServersAndRegionsInTransition(deadServers);
-
-    LOG.info("Joined the cluster in " + (System.currentTimeMillis()
-      - startTime) + "ms, failover=" + failover);
-  }
-
-  /**
-   * Process all regions that are in transition in zookeeper and also
-   * processes the list of dead servers.
-   * Used by master joining an cluster.  If we figure this is a clean cluster
-   * startup, will assign all user regions.
-   * @param deadServers Set of servers that are offline probably legitimately that were carrying
-   * regions according to a scan of hbase:meta. Can be null.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
-  throws KeeperException, IOException, InterruptedException, CoordinatedStateException {
-    // TODO Needed? List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
-    boolean failover = !serverManager.getDeadServers().isEmpty();
-    if (failover) {
-      // This may not be a failover actually, especially if meta is on this master.
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Found dead servers out on cluster " + serverManager.getDeadServers());
-      }
-      // Check if there are any regions on these servers
-      failover = false;
-      for (ServerName serverName : serverManager.getDeadServers().copyServerNames()) {
-        if (regionStates.getRegionAssignments().values().contains(serverName)) {
-          LOG.debug("Found regions on dead server: " + serverName);
-          failover = true;
-          break;
-        }
-      }
-    }
-    Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
-    if (!failover) {
-      // If any one region except meta is assigned, it's a failover.
-      for (Map.Entry<HRegionInfo, ServerName> en:
-          regionStates.getRegionAssignments().entrySet()) {
-        HRegionInfo hri = en.getKey();
-        if (!hri.isMetaTable()
-            && onlineServers.contains(en.getValue())) {
-          LOG.debug("Found region " + hri + " out on cluster");
-          failover = true;
-          break;
-        }
-      }
-    }
-    if (!failover) {
-      // If any region except meta is in transition on a live server, it's a failover.
-      Set<RegionState> regionsInTransition = regionStates.getRegionsInTransition();
-      if (!regionsInTransition.isEmpty()) {
-        for (RegionState regionState: regionsInTransition) {
-          ServerName serverName = regionState.getServerName();
-          if (!regionState.getRegion().isMetaRegion()
-              && serverName != null && onlineServers.contains(serverName)) {
-            LOG.debug("Found " + regionState + " for region " +
-              regionState.getRegion().getRegionNameAsString() + " for server " +
-                serverName + "in RITs");
-            failover = true;
-            break;
-          }
-        }
-      }
-    }
-    if (!failover) {
-      // If we get here, we have a full cluster restart. It is a failover only
-      // if there are some WALs are not split yet. For meta WALs, they should have
-      // been split already, if any. We can walk through those queued dead servers,
-      // if they don't have any WALs, this restart should be considered as a clean one
-      Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
-      if (!queuedDeadServers.isEmpty()) {
-        Configuration conf = server.getConfiguration();
-        Path walRootDir = FSUtils.getWALRootDir(conf);
-        FileSystem walFs = FSUtils.getWALFileSystem(conf);
-        for (ServerName serverName: queuedDeadServers) {
-          // In the case of a clean exit, the shutdown handler would have presplit any WALs and
-          // removed empty directories.
-          Path walDir = new Path(walRootDir,
-            AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
-          Path splitDir = walDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
-          if (checkWals(walFs, walDir) || checkWals(walFs, splitDir)) {
-            LOG.debug("Found queued dead server " + serverName);
-            failover = true;
-            break;
-          }
-        }
-        if (!failover) {
-          // We figured that it's not a failover, so no need to
-          // work on these re-queued dead servers any more.
-          LOG.info("AM figured that it's not a failover and cleaned up "
-            + queuedDeadServers.size() + " queued dead servers");
-          serverManager.removeRequeuedDeadServers();
-        }
-      }
-    }
-
-    Set<TableName> disabledOrDisablingOrEnabling = null;
-    Map<HRegionInfo, ServerName> allRegions = null;
-
-    if (!failover) {
-      disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
-        TableState.State.DISABLED, TableState.State.DISABLING,
-        TableState.State.ENABLING);
-
-      // Clean re/start, mark all user regions closed before reassignment
-      allRegions = regionStates.closeAllUserRegions(
-        disabledOrDisablingOrEnabling);
-    }
-
-    // Now region states are restored
-    regionStateStore.start();
-
-    if (failover) {
-      if (deadServers != null && !deadServers.isEmpty()) {
-        for (ServerName serverName: deadServers) {
-          if (!serverManager.isServerDead(serverName)) {
-            serverManager.expireServer(serverName); // Let SSH do region re-assign
-          }
-        }
-      }
-      processRegionsInTransition(regionStates.getRegionsInTransition());
-    }
-
-    // Now we can safely claim failover cleanup completed and enable
-    // ServerShutdownHandler for further processing. The nodes (below)
-    // in transition, if any, are for regions not related to those
-    // dead servers at all, and can be done in parallel to SSH.
-    failoverCleanupDone();
-    if (!failover) {
-      // Fresh cluster startup.
-      LOG.info("Clean cluster startup. Don't reassign user regions");
-      assignAllUserRegions(allRegions);
-    } else {
-      LOG.info("Failover! Reassign user regions");
-    }
-    // unassign replicas of the split parents and the merged regions
-    // the daughter replicas are opened in assignAllUserRegions if it was
-    // not already opened.
-    for (HRegionInfo h : replicasToClose) {
-      unassign(h);
-    }
-    replicasToClose.clear();
-    return failover;
-  }
-
-  private boolean checkWals(FileSystem fs, Path dir) throws IOException {
-    if (!fs.exists(dir)) {
-      LOG.debug(dir + " doesn't exist");
-      return false;
-    }
-    if (!fs.getFileStatus(dir).isDirectory()) {
-      LOG.warn(dir + " is not a directory");
-      return false;
-    }
-    FileStatus[] files = FSUtils.listStatus(fs, dir);
-    if (files == null || files.length == 0) {
-      LOG.debug(dir + " has no files");
-      return false;
-    }
-    for (int i = 0; i < files.length; i++) {
-      if (files[i].isFile() && files[i].getLen() > 0) {
-        LOG.debug(dir + " has a non-empty file: " + files[i].getPath());
-        return true;
-      } else if (files[i].isDirectory() && checkWals(fs, files[i].getPath())) {
-        LOG.debug(dir + " is a directory and has a non-empty file: " + files[i].getPath());
-        return true;
-      }
-    }
-    LOG.debug("Found 0 non-empty wal files for :" + dir);
-    return false;
-  }
-
-  /**
-   * When a region is closed, it should be removed from the regionsToReopen
-   * @param hri HRegionInfo of the region which was closed
-   */
-  public void removeClosedRegion(HRegionInfo hri) {
-    if (regionsToReopen.remove(hri.getEncodedName()) != null) {
-      LOG.debug("Removed region from reopening regions because it was closed");
-    }
-  }
-
-  void processFavoredNodesForDaughters(HRegionInfo parent,
-    HRegionInfo regionA, HRegionInfo regionB) throws IOException {
-    if (shouldAssignFavoredNodes(parent)) {
-      List<ServerName> onlineServers = this.serverManager.getOnlineServersList();
-      ((FavoredNodesPromoter) this.balancer).
-          generateFavoredNodesForDaughter(onlineServers, parent, regionA, regionB);
-    }
-  }
-
-  void processFavoredNodesForMerge(HRegionInfo merged, HRegionInfo regionA, HRegionInfo regionB)
-    throws IOException {
-    if (shouldAssignFavoredNodes(merged)) {
-      ((FavoredNodesPromoter)this.balancer).
-        generateFavoredNodesForMergedRegion(merged, regionA, regionB);
-    }
-  }
-
-  /*
-   * Favored nodes should be applied only when FavoredNodes balancer is configured and the region
-   * belongs to a non-system table.
-   */
-  private boolean shouldAssignFavoredNodes(HRegionInfo region) {
-    return this.shouldAssignRegionsWithFavoredNodes
-        && FavoredNodesManager.isFavoredNodeApplicable(region);
-  }
-
-  /**
-   * Marks the region as online.  Removes it from regions in transition and
-   * updates the in-memory assignment information.
-   * <p>
-   * Used when a region has been successfully opened on a region server.
-   * @param regionInfo
-   * @param sn
-   */
-  void regionOnline(HRegionInfo regionInfo, ServerName sn) {
-    regionOnline(regionInfo, sn, HConstants.NO_SEQNUM);
-  }
-
-  void regionOnline(HRegionInfo regionInfo, ServerName sn, long openSeqNum) {
-    numRegionsOpened.incrementAndGet();
-    regionStates.regionOnline(regionInfo, sn, openSeqNum);
-
-    // Remove plan if one.
-    clearRegionPlan(regionInfo);
-    balancer.regionOnline(regionInfo, sn);
-
-    // Tell our listeners that a region was opened
-    sendRegionOpenedNotification(regionInfo, sn);
-  }
-
-  /**
-   * Marks the region as offline.  Removes it from regions in transition and
-   * removes in-memory assignment information.
-   * <p>
-   * Used when a region has been closed and should remain closed.
-   * @param regionInfo
-   */
-  public void regionOffline(final HRegionInfo regionInfo) {
-    regionOffline(regionInfo, null);
-  }
-
-  public void offlineDisabledRegion(HRegionInfo regionInfo) {
-    replicasToClose.remove(regionInfo);
-    regionOffline(regionInfo);
-  }
-
-  // Assignment methods
-
-  /**
-   * Assigns the specified region.
-   * <p>
-   * If a RegionPlan is available with a valid destination then it will be used
-   * to determine what server region is assigned to.  If no RegionPlan is
-   * available, region will be assigned to a random available server.
-   * <p>
-   * Updates the RegionState and sends the OPEN RPC.
-   * <p>
-   * This will only succeed if the region is in transition and in a CLOSED or
-   * OFFLINE state or not in transition, and of course, the
-   * chosen server is up and running (It may have just crashed!).
-   *
-   * @param region server to be assigned
-   */
-  public void assign(HRegionInfo region) {
-    assign(region, false);
-  }
-
-  /**
-   * Use care with forceNewPlan. It could cause double assignment.
-   */
-  public void assign(HRegionInfo region, boolean forceNewPlan) {
-    if (isDisabledorDisablingRegionInRIT(region)) {
-      return;
-    }
-    String encodedName = region.getEncodedName();
-    Lock lock = locker.acquireLock(encodedName);
-    try {
-      RegionState state = forceRegionStateToOffline(region, forceNewPlan);
-      if (state != null) {
-        if (regionStates.wasRegionOnDeadServer(encodedName)) {
-          LOG.info("Skip assigning " + region.getRegionNameAsString()
-            + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
-            + " is dead but not processed yet");
-          return;
-        }
-        assign(state, forceNewPlan);
-      }
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  /**
-   * Bulk assign regions to <code>destination</code>.
-   * @param destination
-   * @param regions Regions to assign.
-   * @return true if successful
-   */
-  boolean assign(final ServerName destination, final List<HRegionInfo> regions)
-    throws InterruptedException {
-    long startTime = EnvironmentEdgeManager.currentTime();
-    try {
-      int regionCount = regions.size();
-      if (regionCount == 0) {
-        return true;
-      }
-      LOG.info("Assigning " + regionCount + " region(s) to " + destination.toString());
-      Set<String> encodedNames = new HashSet<>(regionCount);
-      for (HRegionInfo region : regions) {
-        encodedNames.add(region.getEncodedName());
-      }
-
-      List<HRegionInfo> failedToOpenRegions = new ArrayList<>();
-      Map<String, Lock> locks = locker.acquireLocks(encodedNames);
-      try {
-        Map<String, RegionPlan> plans = new HashMap<>(regionCount);
-        List<RegionState> states = new ArrayList<>(regionCount);
-        for (HRegionInfo region : regions) {
-          String encodedName = region.getEncodedName();
-          if (!isDisabledorDisablingRegionInRIT(region)) {
-            RegionState state = forceRegionStateToOffline(region, false);
-            boolean onDeadServer = false;
-            if (state != null) {
-              if (regionStates.wasRegionOnDeadServer(encodedName)) {
-                LOG.info("Skip assigning " + region.getRegionNameAsString()
-                  + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
-                  + " is dead but not processed yet");
-                onDeadServer = true;
-              } else {
-                RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
-                plans.put(encodedName, plan);
-                states.add(state);
-                continue;
-              }
-            }
-            // Reassign if the region wasn't on a dead server
-            if (!onDeadServer) {
-              LOG.info("failed to force region state to offline, "
-                + "will reassign later: " + region);
-              failedToOpenRegions.add(region); // assign individually later
-            }
-          }
-          // Release the lock, this region is excluded from bulk assign because
-          // we can't update its state, or set its znode to offline.
-          Lock lock = locks.remove(encodedName);
-          lock.unlock();
-        }
-
-        if (server.isStopped()) {
-          return false;
-        }
-
-        // Add region plans, so we can updateTimers when one region is opened so
-        // that unnecessary timeout on RIT is reduced.
-        this.addPlans(plans);
-
-        List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos = new ArrayList<>(states.size());
-        for (RegionState state: states) {
-          HRegionInfo region = state.getRegion();
-          regionStates.updateRegionState(
-            region, State.PENDING_OPEN, destination);
-          List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
-          if (shouldAssignFavoredNodes(region)) {
-            favoredNodes = server.getFavoredNodesManager().getFavoredNodesWithDNPort(region);
-          }
-          regionOpenInfos.add(new Pair<>(region, favoredNodes));
-        }
-
-        // Move on to open regions.
-        try {
-          // Send OPEN RPC. If it fails on a IOE or RemoteException,
-          // regions will be assigned individually.
-          Configuration conf = server.getConfiguration();
-          long maxWaitTime = System.currentTimeMillis() +
-            conf.getLong("hbase.regionserver.rpc.startup.waittime", 60000);
-          for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
-            try {
-              List<RegionOpeningState> regionOpeningStateList = serverManager
-                .sendRegionOpen(destination, regionOpenInfos);
-              for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
-                RegionOpeningState openingState = regionOpeningStateList.get(k);
-                if (openingState != RegionOpeningState.OPENED) {
-                  HRegionInfo region = regionOpenInfos.get(k).getFirst();
-                  LOG.info("Got opening state " + openingState
-                    + ", will reassign later: " + region);
-                  // Failed opening this region, reassign it later
-                  forceRegionStateToOffline(region, true);
-                  failedToOpenRegions.add(region);
-                }
-              }
-              break;
-            } catch (IOException e) {
-              if (e instanceof RemoteException) {
-                e = ((RemoteException)e).unwrapRemoteException();
-              }
-              if (e instanceof RegionServerStoppedException) {
-                LOG.warn("The region server was shut down, ", e);
-                // No need to retry, the region server is a goner.
-                return false;
-              } else if (e instanceof ServerNotRunningYetException) {
-                long now = System.currentTimeMillis();
-                if (now < maxWaitTime) {
-                  if (LOG.isDebugEnabled()) {
-                    LOG.debug("Server is not yet up; waiting up to " +
-                      (maxWaitTime - now) + "ms", e);
-                  }
-                  Thread.sleep(100);
-                  i--; // reset the try count
-                  continue;
-                }
-              } else if (e instanceof java.net.SocketTimeoutException
-                  && this.serverManager.isServerOnline(destination)) {
-                // In case socket is timed out and the region server is still online,
-                // the openRegion RPC could have been accepted by the server and
-                // just the response didn't go through.  So we will retry to
-                // open the region on the same server.
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("Bulk assigner openRegion() to " + destination
-                    + " has timed out, but the regions might"
-                    + " already be opened on it.", e);
-                }
-                // wait and reset the re-try count, server might be just busy.
-                Thread.sleep(100);
-                i--;
-                continue;
-              } else if (e instanceof FailedServerException && i < maximumAttempts) {
-                // In case the server is in the failed server list, no point to
-                // retry too soon. Retry after the failed_server_expiry time
-                long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-                  RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug(destination + " is on failed server list; waiting "
-                    + sleepTime + "ms", e);
-                }
-                Thread.sleep(sleepTime);
-                continue;
-              }
-              throw e;
-            }
-          }
-        } catch (IOException e) {
-          // Can be a socket timeout, EOF, NoRouteToHost, etc
-          LOG.info("Unable to communicate with " + destination
-            + " in order to assign regions, ", e);
-          for (RegionState state: states) {
-            HRegionInfo region = state.getRegion();
-            forceRegionStateToOffline(region, true);
-          }
-          return false;
-        }
-      } finally {
-        for (Lock lock : locks.values()) {
-          lock.unlock();
-        }
-      }
-
-      if (!failedToOpenRegions.isEmpty()) {
-        for (HRegionInfo region : failedToOpenRegions) {
-          if (!regionStates.isRegionOnline(region)) {
-            invokeAssign(region);
-          }
-        }
-      }
-
-      // wait for assignment completion
-      ArrayList<HRegionInfo> userRegionSet = new ArrayList<>(regions.size());
-      for (HRegionInfo region: regions) {
-        if (!region.getTable().isSystemTable()) {
-          userRegionSet.add(region);
-        }
-      }
-      if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
-            System.currentTimeMillis())) {
-        LOG.debug("some user regions are still in transition: " + userRegionSet);
-      }
-      LOG.debug("Bulk assigning done for " + destination);
-      return true;
-    } finally {
-      metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTime() - startTime);
-    }
-  }
-
-  /**
-   * Send CLOSE RPC if the server is online, otherwise, offline the region.
-   *
-   * The RPC will be sent only to the region sever found in the region state
-   * if it is passed in, otherwise, to the src server specified. If region
-   * state is not specified, we don't update region state at all, instead
-   * we just send the RPC call. This is useful for some cleanup without
-   * messing around the region states (see handleRegion, on region opened
-   * on an unexpected server scenario, for an example)
-   */
-  private void unassign(final HRegionInfo region,
-      final ServerName server, final ServerName dest) {
-    for (int i = 1; i <= this.maximumAttempts; i++) {
-      if (this.server.isStopped() || this.server.isAborted()) {
-        LOG.debug("Server stopped/aborted; skipping unassign of " + region);
-        return;
-      }
-      if (!serverManager.isServerOnline(server)) {
-        LOG.debug("Offline " + region.getRegionNameAsString()
-          + ", no need to unassign since it's on a dead server: " + server);
-        regionStates.updateRegionState(region, State.OFFLINE);
-        return;
-      }
-      try {
-        // Send CLOSE RPC
-        if (serverManager.sendRegionClose(server, region, dest)) {
-          LOG.debug("Sent CLOSE to " + server + " for region " +
-            region.getRegionNameAsString());
-          return;
-        }
-        // This never happens. Currently regionserver close always return true.
-        // Todo; this can now happen (0.96) if there is an exception in a coprocessor
-        LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
-          region.getRegionNameAsString());
-      } catch (Throwable t) {
-        long sleepTime = 0;
-        Configuration conf = this.server.getConfiguration();
-        if (t instanceof RemoteException) {
-          t = ((RemoteException)t).unwrapRemoteException();
-        }
-        if (t instanceof RegionServerAbortedException
-            || t instanceof RegionServerStoppedException
-            || t instanceof ServerNotRunningYetException) {
-          // RS is aborting, we cannot offline the region since the region may need to do WAL
-          // recovery. Until we see  the RS expiration, we should retry.
-          sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-            RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
-
-        } else if (t instanceof NotServingRegionException) {
-          LOG.debug("Offline " + region.getRegionNameAsString()
-            + ", it's not any more on " + server, t);
-          regionStates.updateRegionState(region, State.OFFLINE);
-          return;
-        } else if (t instanceof FailedServerException && i < maximumAttempts) {
-          // In case the server is in the failed server list, no point to
-          // retry too soon. Retry after the failed_server_expiry time
-          sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-          RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(server + " is on failed server list; waiting " + sleepTime + "ms", t);
-          }
-       }
-       try {
-         if (sleepTime > 0) {
-           Thread.sleep(sleepTime);
-         }
-       } catch (InterruptedException ie) {
-         LOG.warn("Interrupted unassign " + region.getRegionNameAsString(), ie);
-         Thread.currentThread().interrupt();
-         regionStates.updateRegionState(region, State.FAILED_CLOSE);
-         return;
-       }
-       LOG.info("Server " + server + " returned " + t + " for "
-         + region.getRegionNameAsString() + ", try=" + i
-         + " of " + this.maximumAttempts, t);
-      }
-    }
-    // Run out of attempts
-    regionStates.updateRegionState(region, State.FAILED_CLOSE);
-  }
-
-  /**
-   * Set region to OFFLINE unless it is opening and forceNewPlan is false.
-   */
-  private RegionState forceRegionStateToOffline(
-      final HRegionInfo region, final boolean forceNewPlan) {
-    RegionState state = regionStates.getRegionState(region);
-    if (state == null) {
-      LOG.warn("Assigning but not in region states: " + region);
-      state = regionStates.createRegionState(region);
-    }
-
-    if (forceNewPlan && LOG.isDebugEnabled()) {
-      LOG.debug("Force region state offline " + state);
-    }
-
-    switch (state.getState()) {
-    case OPEN:
-    case OPENING:
-    case PENDING_OPEN:
-    case CLOSING:
-    case PENDING_CLOSE:
-      if (!forceNewPlan) {
-        LOG.debug("Skip assigning " +
-          region + ", it is already " + state);
-        return null;
-      }
-    case FAILED_CLOSE:
-    case FAILED_OPEN:
-      regionStates.updateRegionState(region, State.PENDING_CLOSE);
-      unassign(region, state.getServerName(), null);
-      state = regionStates.getRegionState(region);
-      if (!state.isOffline() && !state.isClosed()) {
-        // If the region isn't offline, we can't re-assign
-        // it now. It will be assigned automatically after
-        // the regionserver reports it's closed.
-        return null;
-      }
-    case OFFLINE:
-    case CLOSED:
-      break;
-    default:
-      LOG.error("Trying to assign region " + region
-        + ", which is " + state);
-      return null;
-    }
-    return state;
-  }
-
-  /**
-   * Caller must hold lock on the passed <code>state</code> object.
-   * @param state
-   * @param forceNewPlan
-   */
-  private void assign(RegionState state, boolean forceNewPlan) {
-    long startTime = EnvironmentEdgeManager.currentTime();
-    try {
-      Configuration conf = server.getConfiguration();
-      RegionPlan plan = null;
-      long maxWaitTime = -1;
-      HRegionInfo region = state.getRegion();
-      Throwable previousException = null;
-      for (int i = 1; i <= maximumAttempts; i++) {
-        if (server.isStopped() || server.isAborted()) {
-          LOG.info("Skip assigning " + region.getRegionNameAsString()
-            + ", the server is stopped/aborted");
-          return;
-        }
-
-        if (plan == null) { // Get a server for the region at first
-          try {
-            plan = getRegionPlan(region, forceNewPlan);
-          } catch (HBaseIOException e) {
-            LOG.warn("Failed to get region plan", e);
-          }
-        }
-
-        if (plan == null) {
-          LOG.warn("Unable to determine a plan to assign " + region);
-
-          // For meta region, we have to keep retrying until succeeding
-          if (region.isMetaRegion()) {
-            if (i == maximumAttempts) {
-              i = 0; // re-set attempt count to 0 for at least 1 retry
-
-              LOG.warn("Unable to determine a plan to assign a hbase:meta region " + region +
-                " after maximumAttempts (" + this.maximumAttempts +
-                "). Reset attempts count and continue retrying.");
-            }
-            waitForRetryingMetaAssignment();
-            continue;
-          }
-
-          regionStates.updateRegionState(region, State.FAILED_OPEN);
-          return;
-        }
-        LOG.info("Assigning " + region.getRegionNameAsString() +
-            " to " + plan.getDestination());
-        // Transition RegionState to PENDING_OPEN
-       regionStates.updateRegionState(region,
-          State.PENDING_OPEN, plan.getDestination());
-
-        boolean needNewPlan = false;
-        final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
-            " to " + plan.getDestination();
-        try {
-          List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
-          if (shouldAssignFavoredNodes(region)) {
-            favoredNodes = server.getFavoredNodesManager().getFavoredNodesWithDNPort(region);
-          }
-          serverManager.sendRegionOpen(plan.getDestination(), region, favoredNodes);
-          return; // we're done
-        } catch (Throwable t) {
-          if (t instanceof RemoteException) {
-            t = ((RemoteException) t).unwrapRemoteException();
-          }
-          previousException = t;
-
-          // Should we wait a little before retrying? If the server is starting it's yes.
-          boolean hold = (t instanceof ServerNotRunningYetException);
-
-          // In case socket is timed out and the region server is still online,
-          // the openRegion RPC could have been accepted by the server and
-          // just the response didn't go through.  So we will retry to
-          // open the region on the same server.
-          boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
-              && this.serverManager.isServerOnline(plan.getDestination()));
-
-          if (hold) {
-            LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
-              "try=" + i + " of " + this.maximumAttempts, t);
-
-            if (maxWaitTime < 0) {
-              maxWaitTime = EnvironmentEdgeManager.currentTime()
-                + this.server.getConfiguration().getLong(
-                  "hbase.regionserver.rpc.startup.waittime", 60000);
-            }
-            try {
-              long now = EnvironmentEdgeManager.currentTime();
-              if (now < maxWaitTime) {
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("Server is not yet up; waiting up to "
-                    + (maxWaitTime - now) + "ms", t);
-                }
-                Thread.sleep(100);
-                i--; // reset the try count
-              } else {
-                LOG.debug("Server is not up for a while; try a new one", t);
-                needNewPlan = true;
-              }
-            } catch (InterruptedException ie) {
-              LOG.warn("Failed to assign "
-                  + region.getRegionNameAsString() + " since interrupted", ie);
-              regionStates.updateRegionState(region, State.FAILED_OPEN);
-              Thread.currentThread().interrupt();
-              return;
-            }
-          } else if (retry) {
-            i--; // we want to retry as many times as needed as long as the RS is not dead.
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(assignMsg + ", trying to assign to the same region server due ", t);
-            }
-          } else {
-            needNewPlan = true;
-            LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
-                " try=" + i + " of " + this.maximumAttempts, t);
-          }
-        }
-
-        if (i == this.maximumAttempts) {
-          // For meta region, we have to keep retrying until succeeding
-          if (region.isMetaRegion()) {
-            i = 0; // re-set attempt count to 0 for at least 1 retry
-            LOG.warn(assignMsg +
-                ", trying to assign a hbase:meta region reached to maximumAttempts (" +
-                this.maximumAttempts + ").  Reset attempt counts and continue retrying.");
-            waitForRetryingMetaAssignment();
-          }
-          else {
-            // Don't reset the region state or get a new plan any more.
-            // This is the last try.
-            continue;
-          }
-        }
-
-        // If region opened on destination of present plan, reassigning to new
-        // RS may cause double assignments. In case of RegionAlreadyInTransitionException
-        // reassigning to same RS.
-        if (needNewPlan) {
-          // Force a new plan and reassign. Will return null if no servers.
-          // The new plan could be the same as the existing plan since we don't
-          // exclude the server of the original plan, which should not be
-          // excluded since it could be the only server up now.
-          RegionPlan newPlan = null;
-          try {
-            newPlan = getRegionPlan(region, true);
-          } catch (HBaseIOException e) {
-            LOG.warn("Failed to get region plan", e);
-          }
-          if (newPlan == null) {
-            regionStates.updateRegionState(region, State.FAILED_OPEN);
-            LOG.warn("Unable to find a viable location to assign region " +
-                region.getRegionNameAsString());
-            return;
-          }
-
-          if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
-            // Clean out plan we failed execute and one that doesn't look like it'll
-            // succeed anyways; we need a new plan!
-            // Transition back to OFFLINE
-            regionStates.updateRegionState(region, State.OFFLINE);
-            plan = newPlan;
-          } else if(plan.getDestination().equals(newPlan.getDestination()) &&
-              previousException instanceof FailedServerException) {
-            try {
-              LOG.info("Trying to re-assign " + region.getRegionNameAsString() +
-                " to the same failed server.");
-              Thread.sleep(1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-                RpcClient.FAILED_SERVER_EXPIRY_DEFAULT));
-            } catch (InterruptedException ie) {
-              LOG.warn("Failed to assign "
-                  + region.getRegionNameAsString() + " since interrupted", ie);
-              regionStates.updateRegionState(region, State.FAILED_OPEN);
-              Thread.currentThread().interrupt();
-              return;
-            }
-          }
-        }
-      }
-      // Run out of attempts
-      regionStates.updateRegionState(region, State.FAILED_OPEN);
-    } finally {
-      metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTime() - startTime);
-    }
-  }
-
-  private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
-    if (this.tableStateManager.isTableState(region.getTable(),
-            TableState.State.DISABLED,
-            TableState.State.DISABLING) || replicasToClose.contains(region)) {
-      LOG.info("Table " + region.getTable() + " is disabled or disabling;"
-        + " skipping assign of " + region.getRegionNameAsString());
-      offlineDisabledRegion(region);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * @param region the region to assign
-   * @param forceNewPlan If true, then if an existing plan exists, a new plan
-   * will be generated.
-   * @return Plan for passed <code>region</code> (If none currently, it creates one or
-   * if no servers to assign, it returns null).
-   */
-  private RegionPlan getRegionPlan(final HRegionInfo region,
-      final boolean forceNewPlan) throws HBaseIOException {
-    // Pickup existing plan or make a new one
-    final String encodedName = region.getEncodedName();
-    final List<ServerName> destServers =
-      serverManager.createDestinationServersList();
-
-    if (destServers.isEmpty()){
-      LOG.warn("Can't move " + encodedName +
-        ", there is no destination server available.");
-      return null;
-    }
-
-    RegionPlan randomPlan = null;
-    boolean newPlan = false;
-    RegionPlan existingPlan;
-
-    synchronized (this.regionPlans) {
-      existingPlan = this.regionPlans.get(encodedName);
-
-      if (existingPlan != null && existingPlan.getDestination() != null) {
-        LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
-          + " destination server is " + existingPlan.getDestination() +
-            " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
-      }
-
-      if (forceNewPlan
-          || existingPlan == null
-          || existingPlan.getDestination() == null
-          || !destServers.contains(existingPlan.getDestination())) {
-        newPlan = true;
-        try {
-          randomPlan = new RegionPlan(region, null,
-              balancer.randomAssignment(region, destServers));
-        } catch (IOException ex) {
-          LOG.warn("Failed to create new plan.",ex);
-          return null;
-        }
-        this.regionPlans.put(encodedName, randomPlan);
-      }
-    }
-
-    if (newPlan) {
-      if (randomPlan.getDestination() == null) {
-        LOG.warn("Can't find a destination for " + encodedName);
-        return null;
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("No previous transition plan found (or ignoring " +
-          "an existing plan) for " + region.getRegionNameAsString() +
-          "; generated random plan=" + randomPlan + "; " + destServers.size() +
-          " (online=" + serverManager.getOnlineServers().size() +
-          ") available servers, forceNewPlan=" + forceNewPlan);
-      }
-      return randomPlan;
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using pre-existing plan for " +
-        region.getRegionNameAsString() + "; plan=" + existingPlan);
-    }
-    return existingPlan;
-  }
-
-  /**
-   * Wait for some time before retrying meta table region assignment
-   */
-  private void waitForRetryingMetaAssignment() {
-    try {
-      Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
-    } catch (InterruptedException e) {
-      LOG.error("Got exception while waiting for hbase:meta assignment");
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  /**
-   * Unassigns the specified region.
-   * <p>
-   * Updates the RegionState and sends the CLOSE RPC unless region is being
-   * split by regionserver; then the unassign fails (silently) because we
-   * presume the region being unassigned no longer exists (its been split out
-   * of existence). TODO: What to do if split fails and is rolled back and
-   * parent is revivified?
-   * <p>
-   * If a RegionPlan is already set, it will remain.
-   *
-   * @param region server to be unassigned
-   */
-  public void unassign(HRegionInfo region) {
-    unassign(region, null);
-  }
-
-
-  /**
-   * Unassigns the specified region.
-   * <p>
-   * Updates the RegionState and sends the CLOSE RPC unless region is being
-   * split by regionserver; then the unassign fails (silently) because we
-   * presume the region being unassigned no longer exists (its been split out
-   * of existence). TODO: What to do if split fails and is rolled back and
-   * parent is revivified?
-   * <p>
-   * If a RegionPlan is already set, it will remain.
-   *
-   * @param region server to be unassigned
-   * @param dest the destination server of the region
-   */
-  public void unassign(HRegionInfo region, ServerName dest) {
-    // TODO: Method needs refactoring.  Ugly buried returns throughout.  Beware!
-    LOG.debug("Starting unassign of " + region.getRegionNameAsString()
-      + " (offlining), current state: " + regionStates.getRegionState(region));
-
-    String encodedName = region.getEncodedName();
-    // Grab the state of this region and synchronize on it
-    // We need a lock here as we're going to do a put later and we don't want multiple states
-    //  creation
-    ReentrantLock lock = locker.acquireLock(encodedName);
-    RegionState state = regionStates.getRegionTransitionState(encodedName);
-    try {
-      if (state == null || state.isFailedClose()) {
-        if (state == null) {
-          // Region is not in transition.
-          // We can unassign it only if it's not SPLIT/MERGED.
-          state = regionStates.getRegionState(encodedName);
-          if (state != null && state.isUnassignable()) {
-            LOG.info("Attempting to unassign " + state + ", ignored");
-            // Offline region will be reassigned below
-            return;
-          }
-          if (state == null || state.getServerName() == null) {
-            // We don't know where the region is, offline it.
-            // No need to send CLOSE RPC
-            LOG.warn("Attempting to unassign a region not in RegionStates "
-              + region.getRegionNameAsString() + ", offlined");
-            regionOffline(region);
-            return;
-          }
-        }
-        state = regionStates.updateRegionState(
-          region, State.PENDING_CLOSE);
-      } else if (state.isFailedOpen()) {
-        // The region is not open yet
-        regionOffline(region);
-        return;
-      } else {
-        LOG.debug("Attempting to unassign " +
-          region.getRegionNameAsString() + " but it is " +
-          "already in transition (" + state.getState());
-        return;
-      }
-
-      unassign(region, state.getServerName(), dest);
-    } finally {
-      lock.unlock();
-
-      // Region is expected to be reassigned afterwards
-      if (!replicasToClose.contains(region)
-          && regionStates.isRegionInState(region, State.OFFLINE)) {
-        assign(region);
-      }
-    }
-  }
-
-  /**
-   * Used by unit tests. Return the number of regions opened so far in the life
-   * of the master. Increases by one every time the master opens a region
-   * @return the counter value of the number of regions opened so far
-   */
-  public int getNumRegionsOpened() {
-    return numRegionsOpened.get();
-  }
-
-  /**
-   * Waits until the specified region has completed assignment.
-   * <p>
-   * If the region is already assigned, returns immediately.  Otherwise, method
-   * blocks until the region is assigned.
-   * @param regionInfo region to wait on assignment for
-   * @return true if the region is assigned false otherwise.
-   * @throws InterruptedException
-   */
-  public boolean waitForAssignment(HRegionInfo regionInfo)
-      throws InterruptedException {
-    ArrayList<HRegionInfo> regionSet = new ArrayList<>(1);
-    regionSet.add(regionInfo);
-    return waitForAssignment(regionSet, true, Long.MAX_VALUE);
-  }
-
-  /**
-   * Waits until the specified region has completed assignment, or the deadline is reached.
-   */
-  protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
-      final boolean waitTillAllAssigned, final int reassigningRegions,
-      final long minEndTime) throws InterruptedException {
-    long deadline = minEndTime + bulkPerRegionOpenTimeGuesstimate * (reassigningRegions + 1);
-    if (deadline < 0) { // Overflow
-      deadline = Long.MAX_VALUE; // wait forever
-    }
-    return waitForAssignment(regionSet, waitTillAllAssigned, deadline);
-  }
-
-  /**
-   * Waits until the specified region has completed assignment, or the deadline is reached.
-   * @param regionSet set of region to wait on. the set is modified and the assigned regions removed
-   * @param waitTillAllAssigned true if we should wait all the regions to be assigned
-   * @param deadline the timestamp after which the wait is aborted
-   * @return true if all the regions are assigned false otherwise.
-   * @throws InterruptedException
-   */
-  protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
-      final boolean waitTillAllAssigned, final long deadline) throws InterruptedException {
-    // We're not synchronizing on regionsInTransition now because we don't use any iterator.
-    while (!regionSet.isEmpty() && !server.isStopped() && deadline > System.currentTimeMillis()) {
-      int failedOpenCount = 0;
-      Iterator<HRegionInfo> regionInfoIterator = regionSet.iterator();
-      while (regionInfoIterator.hasNext()) {
-        HRegionInfo hri = regionInfoIterator.next();
-        if (regionStates.isRegionOnline(hri) || regionStates.isRegionInState(hri,
-            State.SPLITTING, State.SPLIT, State.MERGING, State.MERGED)) {
-          regionInfoIterator.remove();
-        } else if (regionStates.isRegionInState(hri, State.FAILED_OPEN)) {
-          failedOpenCount++;
-        }
-      }
-      if (!waitTillAllAssigned) {
-        // No need to wait, let assignment going on asynchronously
-        break;
-      }
-      if (!regionSet.isEmpty()) {
-        if (failedOpenCount == regionSet.size()) {
-          // all the regions we are waiting had an error on open.
-          break;
-        }
-        regionStates.waitForUpdate(100);
-      }
-    }
-    return regionSet.isEmpty();
-  }
-
-  /**
-   * Assigns the hbase:meta region or a replica.
-   * <p>
-   * Assumes that hbase:meta is currently closed and is not being actively served by
-   * any RegionServer.
-   * @param hri TODO
-   */
-  public void assignMeta(HRegionInfo hri) throws KeeperException {
-    regionStates.updateRegionState(hri, State.OFFLINE);
-    assign(hri);
-  }
-
-  /**
-   * Assigns specified regions retaining assignments, if any.
-   * <p>
-   * This is a synchronous call and will return once every region has been
-   * assigned.  If anything fails, an exception is thrown
-   * @throws InterruptedException
-   * @throws IOException
-   */
-  public void assign(Map<HRegionInfo, ServerName> regions)
-        throws IOException, InterruptedException {
-    if (regions == null || regions.isEmpty()) {
-      return;
-    }
-    List<ServerName> servers = serverManager.createDestinationServersList();
-    if (servers == null || servers.isEmpty()) {
-      throw new IOException("Found no destination server to assign region(s)");
-    }
-
-    // Reuse existing assignment info
-    Map<ServerName, List<HRegionInfo>> bulkPlan =
-      balancer.retainAssignment(regions, servers);
-    if (bulkPlan == null) {
-      throw new IOException("Unable to determine a plan to assign region(s)");
-    }
-
-    processBogusAssignments(bulkPlan);
-
-    assign(regions.size(), servers.size(),
-      "retainAssignment=true", bulkPlan);
-  }
-
-  /**
-   * Assigns specified regions round robin, if any.
-   * <p>
-   * This is a synchronous call and will return once every region has been
-   * assigned.  If anything fails, an exception is thrown
-   * @throws InterruptedException
-   * @throws IOException
-   */
-  public void assign(List<HRegionInfo> regions)
-        throws IOException, InterruptedException {
-    if (regions == null || regions.isEmpty()) {
-      return;
-    }
-
-    List<ServerName> servers = serverManager.createDestinationServersList();
-    if (servers == null || servers.isEmpty()) {
-      throw new IOException("Found no destination server to assign region(s)");
-    }
-
-    // Generate a round-robin bulk assignment plan
-    Map<ServerName, List<HRegionInfo>> bulkPlan = balancer.roundRobinAssignment(regions, servers);
-    if (bulkPlan == null) {
-      throw new IOException("Unable to determine a plan to assign region(s)");
-    }
-
-    processBogusAssignments(bulkPlan);
-
-    assign(regions.size(), servers.size(), "round-robin=true", bulkPlan);
-  }
-
-  private void assign(int regions, int totalServers,
-      String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
-          throws InterruptedException, IOException {
-
-    int servers = bulkPlan.size();
-    if (servers == 1 || (regions < bulkAssignThresholdRegions
-        && servers < bulkAssignThresholdServers)) {
-
-      // Not use bulk assignment.  This could be more efficient in small
-      // cluster, especially mini cluster for testing, so that tests won't time out
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Not using bulk assignment since we are assigning only " + regions +
-          " region(s) to " + servers + " server(s)");
-      }
-
-      // invoke assignment (async)
-      ArrayList<HRegionInfo> userRegionSet = new ArrayList<>(regions);
-      for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
-        if (!assign(plan.getKey(), plan.getValue()) && !server.isStopped()) {
-          for (HRegionInfo region: plan.getValue()) {
-            if (!regionStates.isRegionOnline(region)) {
-              invokeAssign(region);
-              if (!region.getTable().isSystemTable()) {
-                userRegionSet.add(region);
-              }
-            }
-          }
-        }
-      }
-
-      // wait for assignment completion
-      if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
-            System.currentTimeMillis())) {
-        LOG.debug("some user regions are still in transition: " + userRegionSet);
-      }
-    } else {
-      LOG.info("Bulk assigning " + regions + " region(s) across "
-        + totalServers + " server(s), " + message);
-
-      // Use fixed count thread pool assigning.
-      BulkAssigner ba = new GeneralBulkAssigner(
-        this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
-      ba.bulkAssign();
-      LOG.info("Bulk assigning done");
-    }
-  }
-
-  /**
-   * Assigns all user regions, if any exist.  Used during cluster startup.
-   * <p>
-   * This is a synchronous call and will return once every region has been
-   * assigned.  If anything fails, an exception is thrown and the cluster
-   * should be shutdown.
-   * @throws InterruptedException
-   * @throws IOException
-   */
-  private void assignAllUserRegions(Map<HRegionInfo, ServerName> allRegions)
-      throws IOException, InterruptedException {
-    if (allRegions == null || allRegions.isEmpty()) return;
-
-    // Determine what type of assignment to do on startup
-    boolean retainAssignment = server.getConfiguration().
-      getBoolean("hbase.master.startup.retainassign", true);
-
-    Set<HRegionInfo> regionsFromMetaScan = allRegions.keySet();
-    if (retainAssignment) {
-      assign(allRegions);
-    } else {
-      List<HRegionInfo> regions = new ArrayList<>(regionsFromMetaScan);
-      assign(regions);
-    }
-
-    for (HRegionInfo hri : regionsFromMetaScan) {
-      TableName tableName = hri.getTable();
-      if (!tableStateManager.isTableState(tableName,
-              TableState.State.ENABLED)) {
-        setEnabledTable(tableName);
-      }
-    }
-    // assign all the replicas that were not recorded in the meta
-    assign(replicaRegionsNotRecordedInMeta(regionsFromMetaScan, (MasterServices)server));
-  }
-
-  /**
-   * Get number of replicas of a table
-   */
-  private static int getNumReplicas(MasterServices master, TableName table) {
-    int numReplica = 1;
-    try {
-      HTableDescriptor htd = master.getTableDescriptors().get(table);
-      if (htd == null) {
-        LOG.warn("master can not get TableDescriptor from table '" + table);
-      } else {
-        numReplica = htd.getRegionReplication();
-      }
-    } catch (IOException e){
-      LOG.warn("Couldn't get the replication attribute of the table " + table + " due to "
-          + e.getMessage());
-    }
-    return numReplica;
-  }
-
-  /**
-   * Get a list of replica regions that are:
-   * not recorded in meta yet. We might not have recorded the locations
-   * for the replicas since the replicas may not have been online yet, master restarted
-   * in the middle of assigning, ZK erased, etc.
-   * @param regionsRecordedInMeta the list of regions we know are recorded in meta
-   * either as a default, or, as the location of a replica
-   * @param master
-   * @return list of replica regions
-   * @throws IOException
-   */
-  public static List<HRegionInfo> replicaRegionsNotRecordedInMeta(
-      Set<HRegionInfo> regionsRecordedInMeta, MasterServices master)throws IOException {
-    List<HRegionInfo> regionsNotRecordedInMeta = new ArrayList<>();
-    for (HRegionInfo hri : regionsRecordedInMeta) {
-      TableName table = hri.getTable();
-      if(master.getTableDescriptors().get(table) == null)
-        continue;
-      int  desiredRegionReplication = getNumReplicas(master, table);
-      for (int i = 0; i < desiredRegionReplication; i++) {
-        HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
-        if (regionsRecordedInMeta.contains(replica)) continue;
-        regionsNotRecordedInMeta.add(replica);
-      }
-    }
-    return regionsNotRecordedInMeta;
-  }
-
-  /**
-   * Rebuild the list of user regions and assignment information.
-   * Updates regionstates with findings as we go through list of regions.
-   * @return set of servers not online that hosted some regions according to a scan of hbase:meta
-   * @throws IOException
-   */
-  Set<ServerName> rebuildUserRegions() throws
-          IOException, KeeperException {
-    Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
-            TableState.State.DISABLED, TableState.State.ENABLING);
-
-    Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
-            TableState.State.DISABLED,
-            TableState.State.DISABLING,
-            TableState.State.ENABLING);
-
-    // Region assignment from META
-    List<Result> results = MetaTableAccessor.fullScanRegions(server.getConnection());
-    // Get any new but slow to checkin region server that joined the cluster
-    Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
-    // Set of offline servers to be returned
-    Set<ServerName> offlineServers = new HashSet<>();
-    // Iterate regions in META
-    for (Result result : results) {
-      if (result == null && LOG.isDebugEnabled()){
-        LOG.debug("null result from meta - ignoring but this is strange.");
-        continue;
-      }
-      // keep a track of replicas to close. These were the replicas of the originally
-      // unmerged regions. The master might have closed them before but it mightn't
-      // maybe because it crashed.
-      PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(result);
-      if (p.getFirst() != null && p.getSecond() != null) {
-        int numReplicas = getNumReplicas(server, p.getFirst().getTable());
-        for (HRegionInfo merge : p) {
-          for (int i = 1; i < numReplicas; i++) {
-            replicasToClose.add(RegionReplicaUtil.getRegionInfoForReplica(merge, i));
-          }
-        }
-      }
-      RegionLocations rl =  MetaTableAccessor.getRegionLocations(result);
-      if (rl == null) {
-        continue;
-      }
-      HRegionLocation[] locations = rl.getRegionLocations();
-      if (locations == null) {
-        continue;
-      }
-      for (HRegionLocation hrl : locations) {
-        if (hrl == null) continue;
-        HRegionInfo regionInfo = hrl.getRegionInfo();
-        if (regionInfo == null) continue;
-        int replicaId = regionInfo.getReplicaId();
-        State state = RegionStateStore.getRegionState(result, replicaId);
-        // keep a track of replicas to close. These were the replicas of the split parents
-        // from the previous life of the master. The master should have closed them before
-        // but it couldn't maybe because it crashed
-        if (replicaId == 0 && state.equals(State.SPLIT)) {
-          for (HRegionLocation h : locations) {
-            replicasToClose.add(h.getRegionInfo());
-          }
-        }
-        ServerName lastHost = hrl.getServerName();
-        ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
-        regionStates.createRegionState(regionInfo, state, regionLocation, lastHost);
-        if (!regionStates.isRegionInState(regionInfo, State.OPEN)) {
-          // Region is not open (either offline or in transition), skip
-          continue;
-        }
-        TableName tableName = regionInfo.getTable();
-        if (!onlineServers.contains(regionLocation)) {
-          // Region is located on a server that isn't online
-          offlineServers.add(regionLocation);
-        } else if (!disabledOrEnablingTables.contains(tableName)) {
-          // Region is being served and on an active server
-          // add only if region not in disabled or enabling table
-          regionStates.regionOnline(regionInfo, regionLocation);
-          balancer.regionOnline(regionInfo, regionLocation);
-        }
-        // need to enable the table if not disabled or disabling or enabling
-        // this will be used in rolling restarts
-        if (!disabledOrDisablingOrEnabling.contains(tableName)
-          && !getTableStateManager().isTableState(tableName,
-                TableState.State.ENABLED)) {
-          setEnabledTable(tableName);
-        }
-      }
-    }
-    return offlineServers;
-  }
-
-  /**
-   * Processes list of regions in transition at startup
-   */
-  void processRegionsInTransition(Collection<RegionState> regionsInTransition) {
-    // We need to send RPC call again for PENDING_OPEN/PENDING_CLOSE regions
-    // in case the RPC call is not sent out yet before the master was shut down
-    // since we update the state before we send the RPC call. We can't update
-    // the state after the RPC call. Otherwise, we don't know what's happened
-    // to the region if the master dies right after the RPC call is out.
-    for (RegionState regionState: regionsInTransition) {
-      LOG.info("Processing " + regionState);
-      ServerName serverName = regionState.getServerName();
-      // Server could be null in case of FAILED_OPEN when master cannot find a region plan. In that
-      // case, try assigning it here.
-      if (serverName != null && !serverManager.getOnlineServers().containsKey(serverName)) {
-        LOG.info("Server " + serverName + " isn't online. SSH will handle this");
-        continue; // SSH will handle it
-      }
-      HRegionInfo regionInfo = regionState.getRegion();
-      RegionState.State state = regionState.getState();
-      switch (state) {
-      case CLOSED:
-        invokeAssign(regionState.getRegion());
-        break;
-      case PENDING_OPEN:
-        retrySendRegionOpen(regionState);
-        break;
-      case PENDING_CLOSE:
-        retrySendRegionClose(regionState);
-        break;
-      case FAILED_CLOSE:
-      case FAILED_OPEN:
-        invokeUnAssign(regionInfo);
-        break;
-      default:
-          // No process for other states
-          break;
-      }
-    }
-  }
-
-  /**
-   * At master failover, for pending_open region, make sure
-   * sendRegionOpen RPC call is sent to the target regionserver
-   */
-  private void retrySendRegionOpen(final RegionState regionState) {
-    this.executorService.submit(
-      new EventHandler(server, EventType.M_MASTER_RECOVERY) {
-        @Override
-        public void process() throws IOException {
-          HRegionInfo hri = regionState.getRegion();
-          ServerName serverName = regionState.getServerName();
-          ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
-          try {
-            for (int i = 1; i <= maximumAttempts; i++) {
-              if (!serverManager.isServerOnline(serverName)
-                  || server.isStopped() || server.isAborted()) {
-                return; // No need any more
-              }
-              try {
-                if (!regionState.equals(regionStates.getRegionState(hri))) {
-                  return; // Region is not in the expected state any more
-                }
-                List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
-                if (shouldAssignFavoredNodes(hri)) {
-                  FavoredNodesManager fnm = ((MasterServices)server).getFavoredNodesManager();
-                  favoredNodes = fnm.getFavoredNodesWithDNPort(hri);
-                }
-                serverManager.sendRegionOpen(serverName, hri, favoredNodes);
-                return; // we're done
-              } catch (Throwable t) {
-                if (t instanceof RemoteException) {
-                  t = ((RemoteException) t).unwrapRemoteException();
-                }
-                if (t instanceof FailedServerException && i < maximumAttempts) {
-                  // In case the server is in the failed server list, no point to
-                  // retry too soon. Retry after the failed_server_expiry time
-                  try {
-                    Configuration conf = this.server.getConfiguration();
-                    long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-                      RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
-                    if (LOG.isDebugEnabled()) {
-                      LOG.debug(serverName + " is on failed server list; waiting "
-                        + sleepTime + "ms", t);
-                    }
-                    Thread.sleep(sleepTime);
-                    continue;
-                  } catch (InterruptedException ie) {
-                    LOG.warn("Failed to assign "
-                      + hri.getRegionNameAsString() + " since interrupted", ie);
-                    regionStates.updateRegionState(hri, State.FAILED_OPEN);
-                    Thread.currentThread().interrupt();
-                    return;
-                  }
-                }
-                if (serverManager.isServerOnline(serverName)
-                    && t instanceof java.net.SocketTimeoutException) {
-                  i--; // reset the try count
-                } else {
-                  LOG.info("Got exception in retrying sendRegionOpen for "
-                    + regionState + "; try=" + i + " of " + maximumAttempts, t);
-                }
-                Threads.sleep(100);
-              }
-            }
-            // Run out of attempts
-            regionStates.updateRegionState(hri, State.FAILED_OPEN);
-          } finally {
-            lock.unlock();
-          }
-        }
-      });
-  }
-
-  /**
-   * At master failover, for pending_close region, make sure
-   * sendRegionClose RPC call is sent to the target regionserver
-   */
-  private void retrySendRegionClose(final RegionState regionState) {
-    this.executorService.submit(
-      new EventHandler(server, EventType.M_MASTER_RECOVERY) {
-        @Override
-        public void process() throws IOException {
-          HRegionInfo hri = regionState.getRegion();
-          ServerName serverName = regionState.getServerName();
-          ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
-          try {
-            for (int i = 1; i <= maximumAttempts; i++) {
-              if (!serverManager.isServerOnline(serverName)
-                  || server.isStopped() || server.isAborted()) {
-                return; // No need any more
-              }
-              try {
-                if (!regionState.equals(regionStates.getRegionState(hri))) {
-                  return; // Region is not in the expected state any more
-                }
-                serverManager.sendRegionClose(serverName, hri, null);
-                return; // Done.
-              } catch (Throwable t) {
-                if (t instanceof RemoteException) {
-                  t = ((RemoteException) t).unwrapRemoteException();
-                }
-                if (t instanceof FailedServerException && i < maximumAttempts) {
-                  // In case the server is in the failed server list, no point to
-                  // retry too soon. Retry after the failed_server_expiry time
-                  try {
-                    Configuration conf = this.server.getConfiguration();
-                    long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
-                      RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
-                    if (LOG.isDebugEnabled()) {
-                      LOG.debug(serverName + " is on failed server list; waiting "
-                        + sleepTime + "ms", t);
-                    }
-                    Thread.sleep(sleepTime);
-                    continue;
-                  } catch (InterruptedException ie) {
-                    LOG.warn("Failed to unassign "
-                      + hri.getRegionNameAsString() + " since interrupted", ie);
-                    regionStates.updateRegionState(hri, RegionState.State.FAILED_CLOSE);
-                    Thread.currentThread().interrupt();
-                    return;
-                  }
-                }
-                if (serverManager.isServerOnline(serverName)
-                    && t instanceof java.net.SocketTimeoutException) {
-                  i--; // reset the try count
-                } else {
-                  LOG.info("Got exception in retrying sendRegionClose for "
-                    + regionState + "; try=" + i + " of " + maximumAttempts, t);
-                }
-                Threads.sleep(100);
-              }
-            }
-            // Run out of attempts
-            regionStates.updateRegionState(hri, State.FAILED_CLOSE);
-          } finally {
-            lock.unlock();
-          }
-        }
-      });
-  }
-
-  /**
-   * Set Regions in transitions metrics.
-   * This takes an iterator on the RegionInTransition map (CLSM), and is not synchronized.
-   * This iterator is not fail fast, which may lead to stale read; but that's better than
-   * creating a copy of the map for metrics computation, as this method will be invoked
-   * on a frequent interval.
-   */
-  public void updateRegionsInTransitionMetrics() {
-    long currentTime = System.currentTimeMillis();
-    int totalRITs = 0;
-    int totalRITsOverThreshold = 0;
-    long oldestRITTime = 0;
-    int ritThreshold = this.server.getConfiguration().
-      getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
-    for (RegionState state: regionStates.getRegionsInTransition()) {
-      totalRITs++;
-      long ritTime = currentTime - state.getStamp();
-      if (ritTime > ritThreshold) { // more than the threshold
-        totalRITsOverThreshold++;
-      }
-      if (oldestRITTime < ritTime) {
-        oldestRITTime = ritTime;
-      }
-    }
-    if (this.metricsAssignmentManager != null) {
-      this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
-      this.metricsAssignmentManager.updateRITCount(totalRITs);
-      this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
-    }
-  }
-
-  /**
-   * @param region Region whose plan we are to clear.
-   */
-  private void clearRegionPlan(final HRegionInfo region) {
-    synchronized (this.regionPlans) {
-      this.regionPlans.remove(region.getEncodedName());
-    }
-  }
-
-  /**
-   * Wait on region to clear regions-in-transition.
-   * @param hri Region to wait on.
-   * @throws IOException
-   */
-  public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
-      throws IOException, InterruptedException {
-    waitOnRegionToClearRegionsInTransition(hri, -1L);
-  }
-
-  /**
-   * Wait on region to clear regions-in-transition or time out
-   * @param hri
-   * @param timeOut Milliseconds to wait for current region to be out of transition state.
-   * @return True when a region clears regions-in-transition before timeout otherwise false
-   * @throws InterruptedException
-   */
-  public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
-      throws InterruptedException {
-    if (!regionStates.isRegionInTransition(hri)) {
-      return true;
-    }
-    long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTime()
-        + timeOut;
-    // There is already a timeout monitor on regions in transition so I
-    // should not have to have one here too?
-    LOG.info("Waiting for " + hri.getEncodedName() +
-        " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
-    while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
-      regionStates.waitForUpdate(100);
-      if (EnvironmentEdgeManager.currentTime() > end) {
-        LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
-        return false;
-      }
-    }
-    if (this.server.isStopped()) {
-      LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
-      return false;
-    }
-    return true;
-  }
-
-  void invokeAssign(HRegionInfo regionInfo) {
-    threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
-  }
-
-  void invokeAssignLater(HRegionInfo regionInfo, long sleepMillis) {
-    scheduledThreadPoolExecutor.schedule(new DelayedAssignCallable(
-        new AssignCallable(this, regionInfo)), sleepMillis, TimeUnit.MILLISECONDS);
-  }
-
-  void invokeUnAssign(HRegionInfo regionInfo) {
-    threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
-  }
-
-  public boolean isCarryingMeta(ServerName serverName) {
-    return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
-  }
-
-  public boolean isCarryingMetaReplica(ServerName serverName, int replicaId) {
-    return isCarryingRegion(serverName,
-        RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId));
-  }
-
-  public boolean isCarryingMetaReplica(ServerName serverName, HRegionInfo metaHri) {
-    return isCarryingRegion(serverName, metaHri);
-  }
-
-  /**
-   * Check if the shutdown server carries the specific region.
-   * @return whether the serverName currently hosts the region
-   */
-  private boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
-    RegionState regionState = regionStates.getRegionTransitionState(hri);
-    ServerName transitionAddr = regionState != null? regionState.getServerName(): null;
-    if (transitionAddr != null) {
-      boolean matchTransitionAddr = transitionAddr.equals(serverName);
-      LOG.debug("Checking region=" + hri.getRegionNameAsString()
-        + ", transitioning on server=" + matchTransitionAddr
-        + " server being checked: " + serverName
-        + ", matches=" + matchTransitionAddr);
-      return matchTransitionAddr;
-    }
-
-    ServerName assignedAddr = regionStates.getRegionServerOfRegion(hri);
-    boolean matchAssignedAddr = serverName.equals(assignedAddr);
-    LOG.debug("based on AM, current region=" + hri.getRegionNameAsString()
-      + " is on server=" + assignedAddr + ", server being checked: "
-      + serverName);
-    return matchAssignedAddr;
-  }
-
-  /**
-   * Clean out crashed server removing any assignments.
-   * @param sn Server that went down.
-   * @return list of regions in transition on this server
-   */
-  public List<HRegionInfo> cleanOutCrashedServerReferences(final ServerName sn) {
-    // Clean out any existing assignment plans for this server
-    synchronized (this.regionPlans) {
-      for (Iterator <Map.Entry<String, RegionPlan>> i = this.regionPlans.entrySet().iterator();
-          i.hasNext();) {
-        Map.Entry<String, RegionPlan> e = i.next();
-        ServerName otherSn = e.getValue().getDestination();
-        // The name will be null if the region is planned for a random assign.
-        if (otherSn != null && otherSn.equals(sn)) {
-          // Use iterator's remove else we'll get CME
-          i.remove();
-        }
-      }
-    }
-    List<HRegionInfo> rits = regionStates.serverOffline(sn);
-    for (Iterator<HRegionInfo> it = rits.iterator(); it.hasNext(); ) {
-      HRegionInfo hri = it.next();
-      String encodedName = hri.getEncodedName();
-
-      // We need a lock on the region as we could update it
-      Lock lock = locker.acquireLock(encodedName);
-      try {
-        RegionState regionState = regionStates.getRegionTransitionState(encodedName);
-        if (regionState == null
-            || (regionState.getServerName() != null && !regionState.isOnServer(sn))
-            || !RegionStates.isOneOfStates(regionState, State.PENDING_OPEN,
-                State.OPENING, State.FAILED_OPEN, State.FAILED_CLOSE, State.OFFLINE)) {
-          LOG.info("Skip " + regionState + " since it is not opening/failed_close"
-            + " on the dead server any more: " + sn);
-          it.remove();
-        } else {
-          if 

<TRUNCATED>

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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
index b53ce45..4d45af3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -21,12 +21,9 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -34,17 +31,11 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.BulkAssigner;
 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.TableStateManager;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.Trace;
 
 @InterfaceAudience.Private
 public class DisableTableProcedure
@@ -116,12 +107,8 @@ public class DisableTableProcedure
         setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
         break;
       case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
-        if (markRegionsOffline(env, tableName, true) ==
-            MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-          setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
-        } else {
-          LOG.trace("Retrying later to disable the missing regions");
-        }
+        addChildProcedure(env.getAssignmentManager().createUnassignProcedures(tableName));
+        setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
         break;
       case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
         setTableStateToDisabled(env, tableName);
@@ -290,83 +277,6 @@ public class DisableTableProcedure
   }
 
   /**
-   * Mark regions of the table offline with retries
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @param retryRequired whether to retry if the first run failed
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  protected static MarkRegionOfflineOpResult markRegionsOffline(
-      final MasterProcedureEnv env,
-      final TableName tableName,
-      final Boolean retryRequired) throws IOException {
-    // Dev consideration: add a config to control max number of retry. For now, it is hard coded.
-    int maxTry = (retryRequired ? 10 : 1);
-    MarkRegionOfflineOpResult operationResult =
-        MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
-    do {
-      try {
-        operationResult = markRegionsOffline(env, tableName);
-        if (operationResult == MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-          break;
-        }
-        maxTry--;
-      } catch (Exception e) {
-        LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
-        maxTry--;
-        if (maxTry > 0) {
-          continue; // we still have some retry left, try again.
-        }
-        throw e;
-      }
-    } while (maxTry > 0);
-
-    if (operationResult != MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
-      LOG.warn("Some or all regions of the Table '" + tableName + "' were still online");
-    }
-
-    return operationResult;
-  }
-
-  /**
-   * Mark regions of the table offline
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  private static MarkRegionOfflineOpResult markRegionsOffline(
-      final MasterProcedureEnv env,
-      final TableName tableName) throws IOException {
-    // Get list of online regions that are of this table.  Regions that are
-    // already closed will not be included in this list; i.e. the returned
-    // list is not ALL regions in a table, its all online regions according
-    // to the in-memory state on this master.
-    MarkRegionOfflineOpResult operationResult =
-        MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL;
-    final List<HRegionInfo> regions =
-        env.getMasterServices().getAssignmentManager().getRegionStates()
-            .getRegionsOfTable(tableName);
-    if (regions.size() > 0) {
-      LOG.info("Offlining " + regions.size() + " regions.");
-
-      BulkDisabler bd = new BulkDisabler(env, tableName, regions);
-      try {
-        if (!bd.bulkAssign()) {
-          operationResult = MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
-        }
-      } catch (InterruptedException e) {
-        LOG.warn("Disable was interrupted");
-        // Preserve the interrupt.
-        Thread.currentThread().interrupt();
-        operationResult = MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_INTERRUPTED;
-      }
-    }
-    return operationResult;
-  }
-
-  /**
    * Mark table state to Disabled
    * @param env MasterProcedureEnv
    * @throws IOException
@@ -428,64 +338,4 @@ public class DisableTableProcedure
       }
     }
   }
-
-  /**
-   * Run bulk disable.
-   */
-  private static class BulkDisabler extends BulkAssigner {
-    private final AssignmentManager assignmentManager;
-    private final List<HRegionInfo> regions;
-    private final TableName tableName;
-    private final int waitingTimeForEvents;
-
-    public BulkDisabler(final MasterProcedureEnv env, final TableName tableName,
-        final List<HRegionInfo> regions) {
-      super(env.getMasterServices());
-      this.assignmentManager = env.getMasterServices().getAssignmentManager();
-      this.tableName = tableName;
-      this.regions = regions;
-      this.waitingTimeForEvents =
-          env.getMasterServices().getConfiguration()
-              .getInt("hbase.master.event.waiting.time", 1000);
-    }
-
-    @Override
-    protected void populatePool(ExecutorService pool) {
-      RegionStates regionStates = assignmentManager.getRegionStates();
-      for (final HRegionInfo region : regions) {
-        if (regionStates.isRegionInTransition(region)
-            && !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) {
-          continue;
-        }
-        pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler", new Runnable() {
-          @Override
-          public void run() {
-            assignmentManager.unassign(region);
-          }
-        }));
-      }
-    }
-
-    @Override
-    protected boolean waitUntilDone(long timeout) throws InterruptedException {
-      long startTime = EnvironmentEdgeManager.currentTime();
-      long remaining = timeout;
-      List<HRegionInfo> regions = null;
-      long lastLogTime = startTime;
-      while (!server.isStopped() && remaining > 0) {
-        Thread.sleep(waitingTimeForEvents);
-        regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
-        long now = EnvironmentEdgeManager.currentTime();
-        // Don't log more than once every ten seconds. Its obnoxious. And only log table regions
-        // if we are waiting a while for them to go down...
-        if (LOG.isDebugEnabled() && ((now - lastLogTime) > 10000)) {
-          lastLogTime = now;
-          LOG.debug("Disable waiting until done; " + remaining + " ms remaining; " + regions);
-        }
-        if (regions.isEmpty()) break;
-        remaining = timeout - (now - startTime);
-      }
-      return regions != null && regions.isEmpty();
-    }
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
new file mode 100644
index 0000000..1478fc7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
@@ -0,0 +1,584 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.CatalogJanitor;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsState;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * The procedure to Merge a region in a table.
+ */
+@InterfaceAudience.Private
+public class DispatchMergingRegionsProcedure
+    extends AbstractStateMachineTableProcedure<DispatchMergingRegionsState> {
+  private static final Log LOG = LogFactory.getLog(DispatchMergingRegionsProcedure.class);
+
+  private final AtomicBoolean aborted = new AtomicBoolean(false);
+  private Boolean traceEnabled;
+  private AssignmentManager assignmentManager;
+  private int timeout;
+  private ServerName regionLocation;
+  private String regionsToMergeListFullName;
+  private String regionsToMergeListEncodedName;
+
+  private TableName tableName;
+  private HRegionInfo [] regionsToMerge;
+  private boolean forcible;
+
+  public DispatchMergingRegionsProcedure() {
+    this.traceEnabled = isTraceEnabled();
+    this.assignmentManager = null;
+    this.timeout = -1;
+    this.regionLocation = null;
+    this.regionsToMergeListFullName = null;
+    this.regionsToMergeListEncodedName = null;
+  }
+
+  public DispatchMergingRegionsProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final HRegionInfo [] regionsToMerge,
+      final boolean forcible) {
+    super(env);
+    this.traceEnabled = isTraceEnabled();
+    this.assignmentManager = getAssignmentManager(env);
+    this.tableName = tableName;
+    // For now, we only merge 2 regions.  It could be extended to more than 2 regions in
+    // the future.
+    assert(regionsToMerge.length == 2);
+    this.regionsToMerge = regionsToMerge;
+    this.forcible = forcible;
+
+    this.timeout = -1;
+    this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
+    this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
+  }
+
+  @Override
+  protected Flow executeFromState(
+      final MasterProcedureEnv env,
+      final DispatchMergingRegionsState state) throws InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case DISPATCH_MERGING_REGIONS_PREPARE:
+        prepareMergeRegion(env);
+        setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PRE_OPERATION);
+        break;
+      case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
+        //Unused for now - reserve to add preMerge coprocessor in the future
+        setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS);
+        break;
+      case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
+        if (MoveRegionsToSameRS(env)) {
+          setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS);
+        } else {
+          LOG.info("Cancel merging regions " + getRegionsToMergeListFullNameString()
+            + ", because can't move them to the same RS");
+          setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
+        }
+        break;
+      case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
+        doMergeInRS(env);
+        setNextState(DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_POST_OPERATION);
+        break;
+      case DISPATCH_MERGING_REGIONS_POST_OPERATION:
+        //Unused for now - reserve to add postCompletedMerge coprocessor in the future
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
+        " in the table " + tableName + " (in state=" + state + ")", e);
+
+      setFailure("master-merge-regions", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(
+      final MasterProcedureEnv env,
+      final DispatchMergingRegionsState state) throws IOException, InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case DISPATCH_MERGING_REGIONS_POST_OPERATION:
+      case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
+        String msg = this + " We are in the " + state + " state."
+            + " It is complicated to rollback the merge operation that region server is working on."
+            + " Rollback is not supported and we should let the merge operation to complete";
+        LOG.warn(msg);
+        // PONR
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      case DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS:
+        break; // nothing to rollback
+      case DISPATCH_MERGING_REGIONS_PRE_OPERATION:
+        break; // nothing to rollback
+      case DISPATCH_MERGING_REGIONS_PREPARE:
+        break; // nothing to rollback
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (Exception e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
+          + getRegionsToMergeListFullNameString() + " in table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected DispatchMergingRegionsState getState(final int stateId) {
+    return DispatchMergingRegionsState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final DispatchMergingRegionsState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected DispatchMergingRegionsState getInitialState() {
+    return DispatchMergingRegionsState.DISPATCH_MERGING_REGIONS_PREPARE;
+  }
+
+  /*
+   * Check whether we are in the state that can be rollback
+   */
+  @Override
+  protected boolean isRollbackSupported(final DispatchMergingRegionsState state) {
+    switch (state) {
+    case DISPATCH_MERGING_REGIONS_POST_OPERATION:
+    case DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS:
+        // It is not safe to rollback if we reach to these states.
+        return false;
+      default:
+        break;
+    }
+    return true;
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.DispatchMergingRegionsStateData.Builder dispatchMergingRegionsMsg =
+        MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
+        .setTableName(ProtobufUtil.toProtoTableName(tableName))
+        .setForcible(forcible);
+    for (HRegionInfo hri: regionsToMerge) {
+      dispatchMergingRegionsMsg.addRegionInfo(HRegionInfo.convert(hri));
+    }
+    dispatchMergingRegionsMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.DispatchMergingRegionsStateData dispatchMergingRegionsMsg =
+        MasterProcedureProtos.DispatchMergingRegionsStateData.parseDelimitedFrom(stream);
+    setUser(MasterProcedureUtil.toUserInfo(dispatchMergingRegionsMsg.getUserInfo()));
+    tableName = ProtobufUtil.toTableName(dispatchMergingRegionsMsg.getTableName());
+
+    assert(dispatchMergingRegionsMsg.getRegionInfoCount() == 2);
+    regionsToMerge = new HRegionInfo[dispatchMergingRegionsMsg.getRegionInfoCount()];
+    for (int i = 0; i < regionsToMerge.length; i++) {
+      regionsToMerge[i] = HRegionInfo.convert(dispatchMergingRegionsMsg.getRegionInfo(i));
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(" regions=");
+    sb.append(getRegionsToMergeListFullNameString());
+    sb.append(" forcible=");
+    sb.append(forcible);
+    sb.append(")");
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    if (!getTableName().isSystemTable() && env.waitInitialized(this)) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    if (env.getProcedureScheduler().waitRegions(this, getTableName(), regionsToMerge)) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), regionsToMerge[0], regionsToMerge[1]);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.MERGE;
+  }
+
+  /**
+   * Prepare merge and do some check
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
+    // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
+    // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+    //
+    CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
+    boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
+    if (regionAHasMergeQualifier
+        || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
+      String msg = "Skip merging regions " + regionsToMerge[0].getRegionNameAsString()
+          + ", " + regionsToMerge[1].getRegionNameAsString() + ", because region "
+          + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
+              .getEncodedName()) + " has merge qualifier";
+      LOG.info(msg);
+      throw new MergeRegionException(msg);
+    }
+
+      RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+      RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
+      RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
+      if (regionStateA == null || regionStateB == null) {
+        throw new UnknownRegionException(
+          regionStateA == null ?
+              regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
+      }
+
+      if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
+        throw new MergeRegionException(
+          "Unable to merge regions not online " + regionStateA + ", " + regionStateB);
+      }
+
+      if (regionsToMerge[0].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
+          regionsToMerge[1].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+        throw new MergeRegionException("Can't merge non-default replicas");
+      }
+
+      if (!forcible && !HRegionInfo.areAdjacent(regionsToMerge[0], regionsToMerge[1])) {
+        throw new MergeRegionException(
+          "Unable to merge not adjacent regions "
+            + regionsToMerge[0].getRegionNameAsString() + ", "
+            + regionsToMerge[1].getRegionNameAsString()
+            + " where forcible = " + forcible);
+      }
+  }
+
+  /**
+   * Move all regions to the same region server
+   * @param env MasterProcedureEnv
+   * @return whether target regions hosted by the same RS
+   * @throws IOException
+   */
+  private boolean MoveRegionsToSameRS(final MasterProcedureEnv env) throws IOException {
+    // Make sure regions are on the same regionserver before send merge
+    // regions request to region server.
+    //
+    boolean onSameRS = isRegionsOnTheSameServer(env);
+    if (!onSameRS) {
+      // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
+      // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+      //
+      RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+      ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
+
+      RegionLoad loadOfRegionA = getRegionLoad(env, regionLocation, regionsToMerge[0]);
+      RegionLoad loadOfRegionB = getRegionLoad(env, regionLocation2, regionsToMerge[1]);
+      if (loadOfRegionA != null && loadOfRegionB != null
+          && loadOfRegionA.getRequestsCount() < loadOfRegionB.getRequestsCount()) {
+        // switch regionsToMerge[0] and regionsToMerge[1]
+        HRegionInfo tmpRegion = this.regionsToMerge[0];
+        this.regionsToMerge[0] = this.regionsToMerge[1];
+        this.regionsToMerge[1] = tmpRegion;
+        ServerName tmpLocation = regionLocation;
+        regionLocation = regionLocation2;
+        regionLocation2 = tmpLocation;
+      }
+
+      long startTime = EnvironmentEdgeManager.currentTime();
+
+      RegionPlan regionPlan = new RegionPlan(regionsToMerge[1], regionLocation2, regionLocation);
+      LOG.info("Moving regions to same server for merge: " + regionPlan.toString());
+      getAssignmentManager(env).moveAsync(regionPlan);
+      do {
+        try {
+          Thread.sleep(20);
+          // Make sure check RIT first, then get region location, otherwise
+          // we would make a wrong result if region is online between getting
+          // region location and checking RIT
+          boolean isRIT = regionStates.isRegionInTransition(regionsToMerge[1]);
+          regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
+          onSameRS = regionLocation.equals(regionLocation2);
+          if (onSameRS || !isRIT) {
+            // Regions are on the same RS, or regionsToMerge[1] is not in
+            // RegionInTransition any more
+            break;
+          }
+        } catch (InterruptedException e) {
+          InterruptedIOException iioe = new InterruptedIOException();
+          iioe.initCause(e);
+          throw iioe;
+        }
+      } while ((EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
+    }
+    return onSameRS;
+  }
+
+  /**
+   * Do the real merge operation in the region server that hosts regions
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void doMergeInRS(final MasterProcedureEnv env) throws IOException {
+    long duration = 0;
+    long startTime = EnvironmentEdgeManager.currentTime();
+    do {
+      try {
+        if (getServerName(env) == null) {
+          // The merge probably already happen. Check
+          RegionState regionState = getAssignmentManager(env).getRegionStates().getRegionState(
+            regionsToMerge[0].getEncodedName());
+          if (regionState.isMerging() || regionState.isMerged()) {
+            LOG.info("Merge regions " +  getRegionsToMergeListEncodedNameString() +
+              " is in progress or completed.  No need to send a new request.");
+          } else {
+            LOG.warn("Cannot sending merge to hosting server of the regions " +
+              getRegionsToMergeListEncodedNameString() + " as the server is unknown");
+          }
+          return;
+        }
+        // TODO: the following RPC call is not idempotent.  Multiple calls (eg. after master
+        // failover, re-execute this step) could result in some exception thrown that does not
+        // paint the correct picture.  This behavior is on-par with old releases.  Improvement
+        // could happen in the future.
+        env.getMasterServices().getServerManager().sendRegionsMerge(
+          getServerName(env),
+          regionsToMerge[0],
+          regionsToMerge[1],
+          forcible,
+          getUser());
+        LOG.info("Sent merge to server " + getServerName(env) + " for region " +
+            getRegionsToMergeListEncodedNameString() + ", forcible=" + forcible);
+        return;
+      } catch (RegionOpeningException roe) {
+        // Do a retry since region should be online on RS immediately
+        LOG.warn("Failed mergering regions in " + getServerName(env) + ", retrying...", roe);
+      } catch (Exception ie) {
+        LOG.warn("Failed sending merge to " + getServerName(env) + " for regions " +
+            getRegionsToMergeListEncodedNameString() + ", forcible=" + forcible, ie);
+        return;
+      }
+    } while ((duration = EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
+
+    // If we reaches here, it means that we get timed out.
+    String msg = "Failed sending merge to " + getServerName(env) + " after " + duration + "ms";
+    LOG.warn(msg);
+    throw new IOException(msg);
+  }
+
+  private RegionLoad getRegionLoad(
+      final MasterProcedureEnv env,
+      final ServerName sn,
+      final HRegionInfo hri) {
+    ServerManager serverManager =  env.getMasterServices().getServerManager();
+    ServerLoad load = serverManager.getLoad(sn);
+    if (load != null) {
+      Map<byte[], RegionLoad> regionsLoad = load.getRegionsLoad();
+      if (regionsLoad != null) {
+        return regionsLoad.get(hri.getRegionName());
+      }
+    }
+    return null;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param env MasterProcedureEnv
+   * @return whether target regions hosted by the same RS
+   */
+  private boolean isRegionsOnTheSameServer(final MasterProcedureEnv env) throws IOException{
+    Boolean onSameRS = true;
+    int i = 0;
+    RegionStates regionStates = getAssignmentManager(env).getRegionStates();
+    regionLocation = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
+    if (regionLocation != null) {
+      for(i = 1; i < regionsToMerge.length; i++) {
+        ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
+        if (regionLocation2 != null) {
+          if (onSameRS) {
+            onSameRS = regionLocation.equals(regionLocation2);
+          }
+        } else {
+          // At least one region is not online, merge will fail, no need to continue.
+          break;
+        }
+      }
+      if (i == regionsToMerge.length) {
+        // Finish checking all regions, return the result;
+        return onSameRS;
+      }
+    }
+
+    // If reaching here, at least one region is not online.
+    String msg = "Skip merging regions " + getRegionsToMergeListFullNameString() +
+        ", because region " + regionsToMerge[i].getEncodedName() + " is not online now.";
+    LOG.warn(msg);
+    throw new IOException(msg);
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param env MasterProcedureEnv
+   * @return assignmentManager
+   */
+  private AssignmentManager getAssignmentManager(final MasterProcedureEnv env) {
+    if (assignmentManager == null) {
+      assignmentManager = env.getMasterServices().getAssignmentManager();
+    }
+    return assignmentManager;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param env MasterProcedureEnv
+   * @return timeout value
+   */
+  private int getTimeout(final MasterProcedureEnv env) {
+    if (timeout == -1) {
+      timeout = env.getMasterConfiguration().getInt(
+        "hbase.master.regionmerge.timeout", regionsToMerge.length * 60 * 1000);
+    }
+    return timeout;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param env MasterProcedureEnv
+   * @return serverName
+   */
+  private ServerName getServerName(final MasterProcedureEnv env) {
+    if (regionLocation == null) {
+      regionLocation =
+          getAssignmentManager(env).getRegionStates().getRegionServerOfRegion(regionsToMerge[0]);
+    }
+    return regionLocation;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param fullName whether return only encoded name
+   * @return region names in a list
+   */
+  private String getRegionsToMergeListFullNameString() {
+    if (regionsToMergeListFullName == null) {
+      StringBuilder sb = new StringBuilder("[");
+      int i = 0;
+      while(i < regionsToMerge.length - 1) {
+        sb.append(regionsToMerge[i].getRegionNameAsString() + ", ");
+        i++;
+      }
+      sb.append(regionsToMerge[i].getRegionNameAsString() + " ]");
+      regionsToMergeListFullName = sb.toString();
+    }
+    return regionsToMergeListFullName;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return encoded region names
+   */
+  private String getRegionsToMergeListEncodedNameString() {
+    if (regionsToMergeListEncodedName == null) {
+      StringBuilder sb = new StringBuilder("[");
+      int i = 0;
+      while(i < regionsToMerge.length - 1) {
+        sb.append(regionsToMerge[i].getEncodedName() + ", ");
+        i++;
+      }
+      sb.append(regionsToMerge[i].getEncodedName() + " ]");
+      regionsToMergeListEncodedName = sb.toString();
+    }
+    return regionsToMergeListEncodedName;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
index 4d67edd..4f4b5b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
@@ -21,34 +21,20 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.BulkAssigner;
-import org.apache.hadoop.hbase.master.GeneralBulkAssigner;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnableTableState;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 
 @InterfaceAudience.Private
 public class EnableTableProcedure
@@ -114,7 +100,7 @@ public class EnableTableProcedure
         setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
         break;
       case ENABLE_TABLE_MARK_REGIONS_ONLINE:
-        markRegionsOnline(env, tableName, true);
+        addChildProcedure(env.getAssignmentManager().createAssignProcedures(tableName));
         setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLED_TABLE_STATE);
         break;
       case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
@@ -287,137 +273,6 @@ public class EnableTableProcedure
   }
 
   /**
-   * Mark offline regions of the table online with retry
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @param retryRequired whether to retry if the first run failed
-   * @throws IOException
-   */
-  protected static void markRegionsOnline(
-      final MasterProcedureEnv env,
-      final TableName tableName,
-      final Boolean retryRequired) throws IOException {
-    // This is best effort approach to make all regions of a table online.  If we fail to do
-    // that, it is ok that the table has some offline regions; user can fix it manually.
-
-    // Dev consideration: add a config to control max number of retry. For now, it is hard coded.
-    int maxTry = (retryRequired ? 10 : 1);
-    boolean done = false;
-
-    do {
-      try {
-        done = markRegionsOnline(env, tableName);
-        if (done) {
-          break;
-        }
-        maxTry--;
-      } catch (Exception e) {
-        LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
-        maxTry--;
-        if (maxTry > 0) {
-          continue; // we still have some retry left, try again.
-        }
-        throw e;
-      }
-    } while (maxTry > 0);
-
-    if (!done) {
-      LOG.warn("Some or all regions of the Table '" + tableName + "' were offline");
-    }
-  }
-
-  /**
-   * Mark offline regions of the table online
-   * @param env MasterProcedureEnv
-   * @param tableName the target table
-   * @return whether the operation is fully completed or being interrupted.
-   * @throws IOException
-   */
-  private static boolean markRegionsOnline(final MasterProcedureEnv env, final TableName tableName)
-      throws IOException {
-    final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
-    final MasterServices masterServices = env.getMasterServices();
-    final ServerManager serverManager = masterServices.getServerManager();
-    boolean done = false;
-    // Get the regions of this table. We're done when all listed
-    // tables are onlined.
-    List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations;
-
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
-      tableRegionsAndLocations =
-          new MetaTableLocator().getMetaRegionsAndLocations(masterServices.getZooKeeper());
-    } else {
-      tableRegionsAndLocations =
-          MetaTableAccessor.getTableRegionsAndLocations(masterServices.getConnection(), tableName);
-    }
-
-    int countOfRegionsInTable = tableRegionsAndLocations.size();
-    Map<HRegionInfo, ServerName> regionsToAssign =
-        regionsToAssignWithServerName(env, tableRegionsAndLocations);
-
-    // need to potentially create some regions for the replicas
-    List<HRegionInfo> unrecordedReplicas =
-        AssignmentManager.replicaRegionsNotRecordedInMeta(new HashSet<>(
-            regionsToAssign.keySet()), masterServices);
-    Map<ServerName, List<HRegionInfo>> srvToUnassignedRegs =
-        assignmentManager.getBalancer().roundRobinAssignment(unrecordedReplicas,
-          serverManager.getOnlineServersList());
-    if (srvToUnassignedRegs != null) {
-      for (Map.Entry<ServerName, List<HRegionInfo>> entry : srvToUnassignedRegs.entrySet()) {
-        for (HRegionInfo h : entry.getValue()) {
-          regionsToAssign.put(h, entry.getKey());
-        }
-      }
-    }
-
-    int offlineRegionsCount = regionsToAssign.size();
-
-    LOG.info("Table '" + tableName + "' has " + countOfRegionsInTable + " regions, of which "
-        + offlineRegionsCount + " are offline.");
-    if (offlineRegionsCount == 0) {
-      return true;
-    }
-
-    List<ServerName> onlineServers = serverManager.createDestinationServersList();
-    Map<ServerName, List<HRegionInfo>> bulkPlan =
-        env.getMasterServices().getAssignmentManager().getBalancer()
-            .retainAssignment(regionsToAssign, onlineServers);
-    if (bulkPlan != null) {
-      LOG.info("Bulk assigning " + offlineRegionsCount + " region(s) across " + bulkPlan.size()
-          + " server(s), retainAssignment=true");
-
-      BulkAssigner ba = new GeneralBulkAssigner(masterServices, bulkPlan, assignmentManager, true);
-      try {
-        if (ba.bulkAssign()) {
-          done = true;
-        }
-      } catch (InterruptedException e) {
-        LOG.warn("Enable operation was interrupted when enabling table '" + tableName + "'");
-        // Preserve the interrupt.
-        Thread.currentThread().interrupt();
-      }
-    } else {
-      LOG.info("Balancer was unable to find suitable servers for table " + tableName
-          + ", leaving unassigned");
-    }
-    return done;
-  }
-
-  /**
-   * Mark regions of the table offline during recovery
-   * @param env MasterProcedureEnv
-   */
-  private void markRegionsOfflineDuringRecovery(final MasterProcedureEnv env) {
-    try {
-      // This is a best effort attempt. We will move on even it does not succeed. We will retry
-      // several times until we giving up.
-      DisableTableProcedure.markRegionsOffline(env, tableName, true);
-    } catch (Exception e) {
-      LOG.debug("Failed to offline all regions of table " + tableName + ". Ignoring", e);
-    }
-  }
-
-  /**
    * Mark table state to Enabled
    * @param env MasterProcedureEnv
    * @throws IOException
@@ -457,31 +312,6 @@ public class EnableTableProcedure
   }
 
   /**
-   * @param regionsInMeta
-   * @return List of regions neither in transition nor assigned.
-   * @throws IOException
-   */
-  private static Map<HRegionInfo, ServerName> regionsToAssignWithServerName(
-      final MasterProcedureEnv env,
-      final List<Pair<HRegionInfo, ServerName>> regionsInMeta) throws IOException {
-    Map<HRegionInfo, ServerName> regionsToAssign = new HashMap<>(regionsInMeta.size());
-    RegionStates regionStates = env.getMasterServices().getAssignmentManager().getRegionStates();
-    for (Pair<HRegionInfo, ServerName> regionLocation : regionsInMeta) {
-      HRegionInfo hri = regionLocation.getFirst();
-      ServerName sn = regionLocation.getSecond();
-      if (regionStates.isRegionOffline(hri)) {
-        regionsToAssign.put(hri, sn);
-      } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Skipping assign for the region " + hri + " during enable table "
-              + hri.getTable() + " because its already in tranition or assigned.");
-        }
-      }
-    }
-    return regionsToAssign;
-  }
-
-  /**
    * Coprocessor Action.
    * @param env MasterProcedureEnv
    * @param state the procedure state

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
index 4b9a7ab..31d05a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
@@ -19,32 +19,19 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.BulkReOpen;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
 /**
  * Helper class for schema change procedures
  */
@@ -60,16 +47,13 @@ public final class MasterDDLOperationHelper {
   public static void deleteColumnFamilyFromFileSystem(
       final MasterProcedureEnv env,
       final TableName tableName,
-      List<HRegionInfo> regionInfoList,
+      final List<HRegionInfo> regionInfoList,
       final byte[] familyName,
-      boolean hasMob) throws IOException {
+      final boolean hasMob) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Removing family=" + Bytes.toString(familyName) + " from table=" + tableName);
     }
-    if (regionInfoList == null) {
-      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, tableName);
-    }
     for (HRegionInfo hri : regionInfoList) {
       // Delete the family directory in FS for all the regions one by one
       mfs.deleteFamilyFromFS(hri, familyName);
@@ -81,77 +65,4 @@ public final class MasterDDLOperationHelper {
       mfs.deleteFamilyFromFS(mobRootDir, mobRegionInfo, familyName);
     }
   }
-
-  /**
-   * Reopen all regions from a table after a schema change operation.
-   **/
-  public static boolean reOpenAllRegions(
-      final MasterProcedureEnv env,
-      final TableName tableName,
-      final List<HRegionInfo> regionInfoList) throws IOException {
-    boolean done = false;
-    LOG.info("Bucketing regions by region server...");
-    List<HRegionLocation> regionLocations = null;
-    Connection connection = env.getMasterServices().getConnection();
-    try (RegionLocator locator = connection.getRegionLocator(tableName)) {
-      regionLocations = locator.getAllRegionLocations();
-    }
-    // Convert List<HRegionLocation> to Map<HRegionInfo, ServerName>.
-    NavigableMap<HRegionInfo, ServerName> hri2Sn = new TreeMap<>();
-    for (HRegionLocation location : regionLocations) {
-      hri2Sn.put(location.getRegionInfo(), location.getServerName());
-    }
-    TreeMap<ServerName, List<HRegionInfo>> serverToRegions = Maps.newTreeMap();
-    List<HRegionInfo> reRegions = new ArrayList<>();
-    for (HRegionInfo hri : regionInfoList) {
-      ServerName sn = hri2Sn.get(hri);
-      // Skip the offlined split parent region
-      // See HBASE-4578 for more information.
-      if (null == sn) {
-        LOG.info("Skip " + hri);
-        continue;
-      }
-      if (!serverToRegions.containsKey(sn)) {
-        LinkedList<HRegionInfo> hriList = Lists.newLinkedList();
-        serverToRegions.put(sn, hriList);
-      }
-      reRegions.add(hri);
-      serverToRegions.get(sn).add(hri);
-    }
-
-    LOG.info("Reopening " + reRegions.size() + " regions on " + serverToRegions.size()
-        + " region servers.");
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    am.setRegionsToReopen(reRegions);
-    BulkReOpen bulkReopen = new BulkReOpen(env.getMasterServices(), serverToRegions, am);
-    while (true) {
-      try {
-        if (bulkReopen.bulkReOpen()) {
-          done = true;
-          break;
-        } else {
-          LOG.warn("Timeout before reopening all regions");
-        }
-      } catch (InterruptedException e) {
-        LOG.warn("Reopen was interrupted");
-        // Preserve the interrupt.
-        Thread.currentThread().interrupt();
-        break;
-      }
-    }
-    return done;
-  }
-
-  /**
-   * Get the region info list of a table from meta if it is not already known by the caller.
-   **/
-  public static List<HRegionInfo> getRegionInfoList(
-    final MasterProcedureEnv env,
-    final TableName tableName,
-    List<HRegionInfo> regionInfoList) throws IOException {
-    if (regionInfoList == null) {
-      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, tableName);
-    }
-    return regionInfoList;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
index c21137d..f815bea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
@@ -29,7 +29,7 @@ public final class MasterProcedureConstants {
 
   /** Number of threads used by the procedure executor */
   public static final String MASTER_PROCEDURE_THREADS = "hbase.master.procedure.threads";
-  public static final int DEFAULT_MIN_MASTER_PROCEDURE_THREADS = 4;
+  public static final int DEFAULT_MIN_MASTER_PROCEDURE_THREADS = 16;
 
   /**
    * Procedure replay sanity check. In case a WAL is missing or unreadable we

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index 2cd5b08..d7a4652 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@@ -93,12 +94,19 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     }
   }
 
+  private final RSProcedureDispatcher remoteDispatcher;
   private final MasterProcedureScheduler procSched;
   private final MasterServices master;
 
   public MasterProcedureEnv(final MasterServices master) {
+    this(master, new RSProcedureDispatcher(master));
+  }
+
+  public MasterProcedureEnv(final MasterServices master,
+      final RSProcedureDispatcher remoteDispatcher) {
     this.master = master;
     this.procSched = new MasterProcedureScheduler(master.getConfiguration());
+    this.remoteDispatcher = remoteDispatcher;
   }
 
   public User getRequestUser() {
@@ -117,6 +125,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.getConfiguration();
   }
 
+  public AssignmentManager getAssignmentManager() {
+    return master.getAssignmentManager();
+  }
+
   public MasterCoprocessorHost getMasterCoprocessorHost() {
     return master.getMasterCoprocessorHost();
   }
@@ -125,6 +137,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return procSched;
   }
 
+  public RSProcedureDispatcher getRemoteDispatcher() {
+    return remoteDispatcher;
+  }
+
   public boolean isRunning() {
     return master.getMasterProcedureExecutor().isRunning();
   }
@@ -138,7 +154,15 @@ public class MasterProcedureEnv implements ConfigurationObserver {
   }
 
   public boolean waitServerCrashProcessingEnabled(Procedure proc) {
-    return procSched.waitEvent(((HMaster)master).getServerCrashProcessingEnabledEvent(), proc);
+    if (master instanceof HMaster) {
+      return procSched.waitEvent(((HMaster)master).getServerCrashProcessingEnabledEvent(), proc);
+    }
+    LOG.warn("server crash processing event on " + master);
+    return false;
+  }
+
+  public boolean waitFailoverCleanup(Procedure proc) {
+    return procSched.waitEvent(master.getAssignmentManager().getFailoverCleanupEvent(), proc);
   }
 
   public void setEventReady(ProcedureEvent event, boolean isReady) {
@@ -153,4 +177,4 @@ public class MasterProcedureEnv implements ConfigurationObserver {
   public void onConfigurationChange(Configuration conf) {
     master.getMasterProcedureExecutor().refreshConfiguration(conf);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index ebf79fa..88c905c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
 import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
 import org.apache.hadoop.hbase.procedure2.LockStatus;
@@ -51,52 +51,51 @@ import org.apache.hadoop.hbase.util.AvlUtil.AvlTreeIterator;
  * This ProcedureScheduler tries to provide to the ProcedureExecutor procedures
  * that can be executed without having to wait on a lock.
  * Most of the master operations can be executed concurrently, if they
- * are operating on different tables (e.g. two create table can be performed
- * at the same, time assuming table A and table B) or against two different servers; say
- * two servers that crashed at about the same time.
+ * are operating on different tables (e.g. two create table procedures can be performed
+ * at the same time) or against two different servers; say two servers that crashed at
+ * about the same time.
  *
- * <p>Each procedure should implement an interface providing information for this queue.
- * for example table related procedures should implement TableProcedureInterface.
- * each procedure will be pushed in its own queue, and based on the operation type
- * we may take smarter decision. e.g. we can abort all the operations preceding
+ * <p>Each procedure should implement an Interface providing information for this queue.
+ * For example table related procedures should implement TableProcedureInterface.
+ * Each procedure will be pushed in its own queue, and based on the operation type
+ * we may make smarter decisions: e.g. we can abort all the operations preceding
  * a delete table, or similar.
  *
  * <h4>Concurrency control</h4>
  * Concurrent access to member variables (tableRunQueue, serverRunQueue, locking, tableMap,
- * serverBuckets) is controlled by schedLock(). That mainly includes:<br>
+ * serverBuckets) is controlled by schedLock(). This mainly includes:<br>
  * <ul>
  *   <li>
- *     {@link #push(Procedure, boolean, boolean)} : A push will add a Queue back to run-queue
+ *     {@link #push(Procedure, boolean, boolean)}: A push will add a Queue back to run-queue
  *     when:
  *     <ol>
- *       <li>queue was empty before push (so must have been out of run-queue)</li>
- *       <li>child procedure is added (which means parent procedure holds exclusive lock, and it
+ *       <li>Queue was empty before push (so must have been out of run-queue)</li>
+ *       <li>Child procedure is added (which means parent procedure holds exclusive lock, and it
  *           must have moved Queue out of run-queue)</li>
  *     </ol>
  *   </li>
  *   <li>
- *     {@link #poll(long)} : A poll will remove a Queue from run-queue when:
+ *     {@link #poll(long)}: A poll will remove a Queue from run-queue when:
  *     <ol>
- *       <li>queue becomes empty after poll</li>
- *       <li>exclusive lock is requested by polled procedure and lock is available (returns the
+ *       <li>Queue becomes empty after poll</li>
+ *       <li>Exclusive lock is requested by polled procedure and lock is available (returns the
  *           procedure)</li>
- *       <li>exclusive lock is requested but lock is not available (returns null)</li>
- *       <li>Polled procedure is child of parent holding exclusive lock, and the next procedure is
+ *       <li>Exclusive lock is requested but lock is not available (returns null)</li>
+ *       <li>Polled procedure is child of parent holding exclusive lock and the next procedure is
  *           not a child</li>
  *     </ol>
  *   </li>
  *   <li>
- *     namespace/table/region locks: Queue is added back to run-queue when lock being released is:
+ *     Namespace/table/region locks: Queue is added back to run-queue when lock being released is:
  *     <ol>
- *       <li>exclusive lock</li>
- *       <li>last shared lock (in case queue was removed because next procedure in queue required
+ *       <li>Exclusive lock</li>
+ *       <li>Last shared lock (in case queue was removed because next procedure in queue required
  *           exclusive lock)</li>
  *     </ol>
  *   </li>
  * </ul>
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   private static final Log LOG = LogFactory.getLog(MasterProcedureScheduler.class);
 
@@ -118,16 +117,16 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * TableQueue with priority 1.
    */
   private static class TablePriorities {
+    final int metaTablePriority;
+    final int userTablePriority;
+    final int sysTablePriority;
+
     TablePriorities(Configuration conf) {
       metaTablePriority = conf.getInt("hbase.master.procedure.queue.meta.table.priority", 3);
       sysTablePriority = conf.getInt("hbase.master.procedure.queue.system.table.priority", 2);
       userTablePriority = conf.getInt("hbase.master.procedure.queue.user.table.priority", 1);
     }
 
-    final int metaTablePriority;
-    final int userTablePriority;
-    final int sysTablePriority;
-
     int getPriority(TableName tableName) {
       if (tableName.equals(TableName.META_TABLE_NAME)) {
         return metaTablePriority;
@@ -773,7 +772,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
           locking.getTableLock(TableName.NAMESPACE_TABLE_NAME);
       namespaceLock.releaseExclusiveLock(procedure);
       int waitingCount = 0;
-      if(systemNamespaceTableLock.releaseSharedLock()) {
+      if (systemNamespaceTableLock.releaseSharedLock()) {
         addToRunQueue(tableRunQueue, getTableQueue(TableName.NAMESPACE_TABLE_NAME));
         waitingCount += wakeWaitingProcedures(systemNamespaceTableLock);
       }
@@ -924,6 +923,12 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * locks.
    */
   private static class SchemaLocking {
+    final Map<ServerName, LockAndQueue> serverLocks = new HashMap<>();
+    final Map<String, LockAndQueue> namespaceLocks = new HashMap<>();
+    final Map<TableName, LockAndQueue> tableLocks = new HashMap<>();
+    // Single map for all regions irrespective of tables. Key is encoded region name.
+    final Map<String, LockAndQueue> regionLocks = new HashMap<>();
+
     private <T> LockAndQueue getLock(Map<T, LockAndQueue> map, T key) {
       LockAndQueue lock = map.get(key);
       if (lock == null) {
@@ -969,11 +974,27 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       regionLocks.clear();
     }
 
-    final Map<ServerName, LockAndQueue> serverLocks = new HashMap<>();
-    final Map<String, LockAndQueue> namespaceLocks = new HashMap<>();
-    final Map<TableName, LockAndQueue> tableLocks = new HashMap<>();
-    // Single map for all regions irrespective of tables. Key is encoded region name.
-    final Map<String, LockAndQueue> regionLocks = new HashMap<>();
+    @Override
+    public String toString() {
+      return "serverLocks=" + filterUnlocked(this.serverLocks) +
+        ", namespaceLocks=" + filterUnlocked(this.namespaceLocks) +
+        ", tableLocks=" + filterUnlocked(this.tableLocks) +
+        ", regionLocks=" + filterUnlocked(this.regionLocks);
+    }
+
+    private String filterUnlocked(Map<?, LockAndQueue> locks) {
+      StringBuilder sb = new StringBuilder("{");
+      int initialLength = sb.length();
+      for (Map.Entry<?, LockAndQueue> entry: locks.entrySet()) {
+        if (!entry.getValue().isLocked()) continue;
+        if (sb.length() > initialLength) sb.append(", ");
+          sb.append(entry.getKey());
+          sb.append("=");
+          sb.append(entry.getValue());
+        }
+        sb.append("}");
+        return sb.toString();
+     }
   }
 
   // ======================================================================
@@ -1057,4 +1078,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return Math.max(1, queue.getPriority() * quantum); // TODO
     }
   }
-}
+
+  /**
+   * For debugging. Expensive.
+    * @throws IOException
+    */
+  @VisibleForTesting
+  public String dumpLocks() throws IOException {
+    // TODO: Refactor so we stream out locks for case when millions; i.e. take a PrintWriter
+    return this.locking.toString();
+  }
+}
\ No newline at end of file


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
new file mode 100644
index 0000000..fdee307
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -0,0 +1,201 @@
+/**
+ * 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.assignment;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.NavigableMap;
+import java.util.SortedSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MasterWalManager;
+import org.apache.hadoop.hbase.master.MockNoopMasterServices;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.security.Superusers;
+
+public class MockMasterServices extends MockNoopMasterServices {
+  private final MasterFileSystem fileSystemManager;
+  private final MasterWalManager walManager;
+  private final AssignmentManager assignmentManager;
+
+  private MasterProcedureEnv procedureEnv;
+  private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
+  private ProcedureStore procedureStore;
+
+  private LoadBalancer balancer;
+  private ServerManager serverManager;
+  // Set of regions on a 'server'. Populated externally. Used in below faking 'cluster'.
+  private final NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers;
+
+  public MockMasterServices(Configuration conf,
+      NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers)
+  throws IOException {
+    super(conf);
+    this.regionsToRegionServers = regionsToRegionServers;
+    Superusers.initialize(conf);
+    this.fileSystemManager = new MasterFileSystem(this);
+    this.walManager = new MasterWalManager(this);
+    this.assignmentManager = new AssignmentManager(this, new MockRegionStateStore(this)) {
+      public boolean isTableEnabled(final TableName tableName) {
+        return true;
+      }
+
+      public boolean isTableDisabled(final TableName tableName) {
+        return false;
+      }
+
+      @Override
+      protected boolean waitServerReportEvent(ServerName serverName, Procedure proc) {
+        // Make a report with current state of the server 'serverName' before we call wait..
+        SortedSet<byte []> regions = regionsToRegionServers.get(serverName);
+        getAssignmentManager().reportOnlineRegions(serverName, 0,
+            regions == null? new HashSet<byte []>(): regions);
+        return super.waitServerReportEvent(serverName, proc);
+      }
+    };
+    this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
+    this.serverManager = new ServerManager(this);
+  }
+
+  public void start(final int numServes, final RSProcedureDispatcher remoteDispatcher)
+      throws IOException {
+    startProcedureExecutor(remoteDispatcher);
+    assignmentManager.start();
+    for (int i = 0; i < numServes; ++i) {
+      serverManager.regionServerReport(
+        ServerName.valueOf("localhost", 100 + i, 1), ServerLoad.EMPTY_SERVERLOAD);
+    }
+  }
+
+  @Override
+  public void stop(String why) {
+    stopProcedureExecutor();
+    this.assignmentManager.stop();
+  }
+
+  private void startProcedureExecutor(final RSProcedureDispatcher remoteDispatcher)
+      throws IOException {
+    final Configuration conf = getConfiguration();
+    final Path logDir = new Path(fileSystemManager.getRootDir(),
+        MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
+
+    //procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
+    //    new MasterProcedureEnv.WALStoreLeaseRecovery(this));
+    procedureStore = new NoopProcedureStore();
+    procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
+
+    procedureEnv = new MasterProcedureEnv(this,
+       remoteDispatcher != null ? remoteDispatcher : new RSProcedureDispatcher(this));
+
+    procedureExecutor = new ProcedureExecutor(conf, procedureEnv, procedureStore,
+        procedureEnv.getProcedureScheduler());
+
+    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
+        Math.max(Runtime.getRuntime().availableProcessors(),
+          MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
+    final boolean abortOnCorruption = conf.getBoolean(
+        MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
+        MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
+    procedureStore.start(numThreads);
+    procedureExecutor.start(numThreads, abortOnCorruption);
+    procedureEnv.getRemoteDispatcher().start();
+  }
+
+  private void stopProcedureExecutor() {
+    if (procedureEnv != null) {
+      procedureEnv.getRemoteDispatcher().stop();
+    }
+
+    if (procedureExecutor != null) {
+      procedureExecutor.stop();
+    }
+
+    if (procedureStore != null) {
+      procedureStore.stop(isAborted());
+    }
+  }
+
+  @Override
+  public boolean isInitialized() {
+    return true;
+  }
+
+  @Override
+  public MasterFileSystem getMasterFileSystem() {
+    return fileSystemManager;
+  }
+
+  @Override
+  public MasterWalManager getMasterWalManager() {
+    return walManager;
+  }
+
+  @Override
+  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return procedureExecutor;
+  }
+
+  @Override
+  public LoadBalancer getLoadBalancer() {
+    return balancer;
+  }
+
+  @Override
+  public ServerManager getServerManager() {
+    return serverManager;
+  }
+
+  @Override
+  public AssignmentManager getAssignmentManager() {
+    return assignmentManager;
+  }
+
+  private static class MockRegionStateStore extends RegionStateStore {
+    public MockRegionStateStore(final MasterServices master) {
+      super(master);
+    }
+
+    public void start() throws IOException {
+    }
+
+    public void stop() {
+    }
+
+    public void updateRegionLocation(final HRegionInfo regionInfo, final State state,
+      final ServerName regionLocation, final ServerName lastHost, final long openSeqNum)
+      throws IOException {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
new file mode 100644
index 0000000..b75f711
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -0,0 +1,567 @@
+/**
+ * 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.assignment;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.NavigableMap;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+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.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+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.rules.TestName;
+import org.junit.rules.TestRule;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestAssignmentManager {
+  private static final Log LOG = LogFactory.getLog(TestAssignmentManager.class);
+  static {
+    Logger.getLogger(MasterProcedureScheduler.class).setLevel(Level.TRACE);
+  }
+  @Rule public TestName name = new TestName();
+  @Rule public final TestRule timeout =
+      CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+        withLookingForStuckThread(true).build();
+
+  private static final int PROC_NTHREADS = 64;
+  private static final int NREGIONS = 1 * 1000;
+  private static final int NSERVERS = Math.max(1, NREGIONS / 200);
+
+  private HBaseTestingUtility UTIL;
+  private MockRSProcedureDispatcher rsDispatcher;
+  private MockMasterServices master;
+  private AssignmentManager am;
+  private NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers =
+      new ConcurrentSkipListMap<ServerName, SortedSet<byte []>>();
+
+  private void setupConfiguration(Configuration conf) throws Exception {
+    FSUtils.setRootDir(conf, UTIL.getDataTestDir());
+    conf.setBoolean(WALProcedureStore.USE_HSYNC_CONF_KEY, false);
+    conf.setInt(WALProcedureStore.SYNC_WAIT_MSEC_CONF_KEY, 10);
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, PROC_NTHREADS);
+    conf.setInt(RSProcedureDispatcher.RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, 1000);
+    conf.setInt(AssignmentManager.ASSIGN_MAX_ATTEMPTS, 5);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    UTIL = new HBaseTestingUtility();
+    setupConfiguration(UTIL.getConfiguration());
+    master = new MockMasterServices(UTIL.getConfiguration(), this.regionsToRegionServers);
+    rsDispatcher = new MockRSProcedureDispatcher(master);
+    master.start(NSERVERS, rsDispatcher);
+    am = master.getAssignmentManager();
+    setUpMeta();
+  }
+
+  private void setUpMeta() throws Exception {
+    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+    am.assignMeta(HRegionInfo.FIRST_META_REGIONINFO);
+    am.wakeMetaLoadedEvent();
+    am.setFailoverCleanupDone(true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    master.stop("tearDown");
+  }
+
+  @Test
+  public void testAssignWithGoodExec() throws Exception {
+    testAssign(new GoodRsExecutor());
+  }
+
+  @Test
+  public void testAssignWithRandExec() throws Exception {
+    final TableName tableName = TableName.valueOf("testAssignWithRandExec");
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    rsDispatcher.setMockRsExecutor(new RandRsExecutor());
+
+    AssignProcedure proc = am.createAssignProcedure(hri, false);
+    //waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false, false)));
+    // TODO
+  }
+
+  @Test
+  public void testSocketTimeout() throws Exception {
+    final TableName tableName = TableName.valueOf(this.name.getMethodName());
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
+    waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
+
+    rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
+    waitOnFuture(submitProcedure(am.createUnassignProcedure(hri, null, false)));
+  }
+
+  @Test
+  public void testServerNotYetRunning() throws Exception {
+    testRetriesExhaustedFailure(TableName.valueOf("testServerNotYetRunning"),
+      new ServerNotYetRunningRsExecutor());
+  }
+
+  private void testRetriesExhaustedFailure(final TableName tableName,
+      final MockRSExecutor executor) throws Exception {
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    // Test Assign operation failure
+    rsDispatcher.setMockRsExecutor(executor);
+    try {
+      waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
+      fail("unexpected assign completion");
+    } catch (RetriesExhaustedException e) {
+      // expected exception
+      LOG.info("expected exception from assign operation: " + e.getMessage(), e);
+    }
+
+    // Assign the region (without problems)
+    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+    waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
+
+    // Test Unassign operation failure
+    rsDispatcher.setMockRsExecutor(executor);
+    waitOnFuture(submitProcedure(am.createUnassignProcedure(hri, null, false)));
+  }
+
+
+  @Test
+  public void testIOExceptionOnAssignment() throws Exception {
+    testFailedOpen(TableName.valueOf("testExceptionOnAssignment"),
+      new FaultyRsExecutor(new IOException("test fault")));
+  }
+
+  @Test
+  public void testDoNotRetryExceptionOnAssignment() throws Exception {
+    testFailedOpen(TableName.valueOf("testDoNotRetryExceptionOnAssignment"),
+      new FaultyRsExecutor(new DoNotRetryIOException("test do not retry fault")));
+  }
+
+  private void testFailedOpen(final TableName tableName,
+      final MockRSExecutor executor) throws Exception {
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    // Test Assign operation failure
+    rsDispatcher.setMockRsExecutor(executor);
+    try {
+      waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
+      fail("unexpected assign completion");
+    } catch (RetriesExhaustedException e) {
+      // expected exception
+      LOG.info("REGION STATE " + am.getRegionStates().getRegionNode(hri));
+      LOG.info("expected exception from assign operation: " + e.getMessage(), e);
+      assertEquals(true, am.getRegionStates().getRegionState(hri).isFailedOpen());
+    }
+  }
+
+  private void testAssign(final MockRSExecutor executor) throws Exception {
+    testAssign(executor, NREGIONS);
+  }
+
+  private void testAssign(final MockRSExecutor executor, final int nregions) throws Exception {
+    rsDispatcher.setMockRsExecutor(executor);
+
+    AssignProcedure[] assignments = new AssignProcedure[nregions];
+
+    long st = System.currentTimeMillis();
+    bulkSubmit(assignments);
+
+    for (int i = 0; i < assignments.length; ++i) {
+      ProcedureTestingUtility.waitProcedure(
+        master.getMasterProcedureExecutor(), assignments[i]);
+      assertTrue(assignments[i].toString(), assignments[i].isSuccess());
+    }
+    long et = System.currentTimeMillis();
+    float sec = ((et - st) / 1000.0f);
+    LOG.info(String.format("[T] Assigning %dprocs in %s (%.2fproc/sec)",
+        assignments.length, StringUtils.humanTimeDiff(et - st), assignments.length / sec));
+  }
+
+  @Test
+  public void testAssignAnAssignedRegion() throws Exception {
+    final TableName tableName = TableName.valueOf("testAssignAnAssignedRegion");
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+
+    final Future<byte[]> futureA = submitProcedure(am.createAssignProcedure(hri, false));
+
+    // wait first assign
+    waitOnFuture(futureA);
+    am.getRegionStates().isRegionInState(hri, State.OPEN);
+    // Second should be a noop. We should recognize region is already OPEN internally
+    // and skip out doing nothing.
+    // wait second assign
+    final Future<byte[]> futureB = submitProcedure(am.createAssignProcedure(hri, false));
+    waitOnFuture(futureB);
+    am.getRegionStates().isRegionInState(hri, State.OPEN);
+    // TODO: What else can we do to ensure just a noop.
+  }
+
+  @Test
+  public void testUnassignAnUnassignedRegion() throws Exception {
+    final TableName tableName = TableName.valueOf("testUnassignAnUnassignedRegion");
+    final HRegionInfo hri = createRegionInfo(tableName, 1);
+
+    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+
+    // assign the region first
+    waitOnFuture(submitProcedure(am.createAssignProcedure(hri, false)));
+
+    final Future<byte[]> futureA = submitProcedure(am.createUnassignProcedure(hri, null, false));
+
+    // Wait first unassign.
+    waitOnFuture(futureA);
+    am.getRegionStates().isRegionInState(hri, State.CLOSED);
+    // Second should be a noop. We should recognize region is already CLOSED internally
+    // and skip out doing nothing.
+    final Future<byte[]> futureB = submitProcedure(am.createUnassignProcedure(hri, null, false));
+    waitOnFuture(futureB);
+    // Ensure we are still CLOSED.
+    am.getRegionStates().isRegionInState(hri, State.CLOSED);
+    // TODO: What else can we do to ensure just a noop.
+  }
+
+  private Future<byte[]> submitProcedure(final Procedure proc) {
+    return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
+  }
+
+  private byte[] waitOnFuture(final Future<byte[]> future) throws Exception {
+    try {
+      return future.get();
+    } catch (ExecutionException e) {
+      throw (Exception)e.getCause();
+    }
+  }
+
+  // ============================================================================================
+  //  Helpers
+  // ============================================================================================
+  private void bulkSubmit(final AssignProcedure[] procs) throws Exception {
+    final Thread[] threads = new Thread[PROC_NTHREADS];
+    for (int i = 0; i < threads.length; ++i) {
+      final int threadId = i;
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          TableName tableName = TableName.valueOf("table-" + threadId);
+          int n = (procs.length / threads.length);
+          int start = threadId * n;
+          int stop = start + n;
+          for (int j = start; j < stop; ++j) {
+            procs[j] = createAndSubmitAssign(tableName, j);
+          }
+        }
+      };
+      threads[i].start();
+    }
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].join();
+    }
+    for (int i = procs.length - 1; i >= 0 && procs[i] == null; --i) {
+      procs[i] = createAndSubmitAssign(TableName.valueOf("table-sync"), i);
+    }
+  }
+
+  private AssignProcedure createAndSubmitAssign(TableName tableName, int regionId) {
+    HRegionInfo hri = createRegionInfo(tableName, regionId);
+    AssignProcedure proc = am.createAssignProcedure(hri, false);
+    master.getMasterProcedureExecutor().submitProcedure(proc);
+    return proc;
+  }
+
+  private UnassignProcedure createAndSubmitUnassign(TableName tableName, int regionId) {
+    HRegionInfo hri = createRegionInfo(tableName, regionId);
+    UnassignProcedure proc = am.createUnassignProcedure(hri, null, false);
+    master.getMasterProcedureExecutor().submitProcedure(proc);
+    return proc;
+  }
+
+  private HRegionInfo createRegionInfo(final TableName tableName, final long regionId) {
+    return new HRegionInfo(tableName,
+      Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
+  }
+
+  private void sendTransitionReport(final ServerName serverName,
+      final RegionInfo regionInfo, final TransitionCode state) throws IOException {
+    ReportRegionStateTransitionRequest.Builder req =
+      ReportRegionStateTransitionRequest.newBuilder();
+    req.setServer(ProtobufUtil.toServerName(serverName));
+    req.addTransition(RegionStateTransition.newBuilder()
+      .addRegionInfo(regionInfo)
+      .setTransitionCode(state)
+      .setOpenSeqNum(1)
+      .build());
+    am.reportRegionStateTransition(req.build());
+  }
+
+  private class NoopRsExecutor implements MockRSExecutor {
+    public ExecuteProceduresResponse sendRequest(ServerName server,
+        ExecuteProceduresRequest request) throws IOException {
+      ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
+      if (request.getOpenRegionCount() > 0) {
+        for (OpenRegionRequest req: request.getOpenRegionList()) {
+          OpenRegionResponse.Builder resp = OpenRegionResponse.newBuilder();
+          for (RegionOpenInfo openReq: req.getOpenInfoList()) {
+            RegionOpeningState state = execOpenRegion(server, openReq);
+            if (state != null) {
+              resp.addOpeningState(state);
+            }
+          }
+          builder.addOpenRegion(resp.build());
+        }
+      }
+      if (request.getCloseRegionCount() > 0) {
+        for (CloseRegionRequest req: request.getCloseRegionList()) {
+          CloseRegionResponse resp = execCloseRegion(server,
+              req.getRegion().getValue().toByteArray());
+          if (resp != null) {
+            builder.addCloseRegion(resp);
+          }
+        }
+      }
+      return ExecuteProceduresResponse.newBuilder().build();
+    }
+
+    protected RegionOpeningState execOpenRegion(ServerName server, RegionOpenInfo regionInfo)
+        throws IOException {
+      return null;
+    }
+
+    protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName)
+        throws IOException {
+      return null;
+    }
+  }
+
+  private class GoodRsExecutor extends NoopRsExecutor {
+    @Override
+    protected RegionOpeningState execOpenRegion(ServerName server, RegionOpenInfo openReq)
+        throws IOException {
+      sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED);
+      // Concurrency?
+      // Now update the state of our cluster in regionsToRegionServers.
+      SortedSet<byte []> regions = regionsToRegionServers.get(server);
+      if (regions == null) {
+        regions = new ConcurrentSkipListSet<byte[]>(Bytes.BYTES_COMPARATOR);
+        regionsToRegionServers.put(server, regions);
+      }
+      HRegionInfo hri = HRegionInfo.convert(openReq.getRegion());
+      if (regions.contains(hri.getRegionName())) {
+        throw new UnsupportedOperationException(hri.getRegionNameAsString());
+      }
+      regions.add(hri.getRegionName());
+      return RegionOpeningState.OPENED;
+    }
+
+    @Override
+    protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName)
+        throws IOException {
+      HRegionInfo hri = am.getRegionInfo(regionName);
+      sendTransitionReport(server, HRegionInfo.convert(hri), TransitionCode.CLOSED);
+      return CloseRegionResponse.newBuilder().setClosed(true).build();
+    }
+  }
+
+  private static class ServerNotYetRunningRsExecutor implements MockRSExecutor {
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      throw new ServerNotRunningYetException("wait on server startup");
+    }
+  }
+
+  private static class FaultyRsExecutor implements MockRSExecutor {
+    private final IOException exception;
+
+    public FaultyRsExecutor(final IOException exception) {
+      this.exception = exception;
+    }
+
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      throw exception;
+    }
+  }
+
+  private class SocketTimeoutRsExecutor extends GoodRsExecutor {
+    private final int maxSocketTimeoutRetries;
+    private final int maxServerRetries;
+
+    private ServerName lastServer;
+    private int sockTimeoutRetries;
+    private int serverRetries;
+
+    public SocketTimeoutRsExecutor(int maxSocketTimeoutRetries, int maxServerRetries) {
+      this.maxServerRetries = maxServerRetries;
+      this.maxSocketTimeoutRetries = maxSocketTimeoutRetries;
+    }
+
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      // SocketTimeoutException should be a temporary problem
+      // unless the server will be declared dead.
+      if (sockTimeoutRetries++ < maxSocketTimeoutRetries) {
+        if (sockTimeoutRetries == 1) assertNotEquals(lastServer, server);
+        lastServer = server;
+        LOG.debug("Socket timeout for server=" + server + " retries=" + sockTimeoutRetries);
+        throw new SocketTimeoutException("simulate socket timeout");
+      } else if (serverRetries++ < maxServerRetries) {
+        LOG.info("Mark server=" + server + " as dead. serverRetries=" + serverRetries);
+        master.getServerManager().moveFromOnlineToDeadServers(server);
+        sockTimeoutRetries = 0;
+        throw new SocketTimeoutException("simulate socket timeout");
+      } else {
+        return super.sendRequest(server, req);
+      }
+    }
+  }
+
+  private class RandRsExecutor extends NoopRsExecutor {
+    private final Random rand = new Random();
+
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      switch (rand.nextInt(5)) {
+        case 0: throw new ServerNotRunningYetException("wait on server startup");
+        case 1: throw new SocketTimeoutException("simulate socket timeout");
+        case 2: throw new RemoteException("java.io.IOException", "unexpected exception");
+      }
+      return super.sendRequest(server, req);
+    }
+
+    @Override
+    protected RegionOpeningState execOpenRegion(ServerName server, RegionOpenInfo openReq)
+        throws IOException {
+      switch (rand.nextInt(6)) {
+        case 0:
+          return OpenRegionResponse.RegionOpeningState.OPENED;
+        case 1:
+          sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED);
+          return OpenRegionResponse.RegionOpeningState.ALREADY_OPENED;
+        case 2:
+          sendTransitionReport(server, openReq.getRegion(), TransitionCode.FAILED_OPEN);
+          return OpenRegionResponse.RegionOpeningState.FAILED_OPENING;
+      }
+      return null;
+    }
+
+    @Override
+    protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName)
+        throws IOException {
+      CloseRegionResponse.Builder resp = CloseRegionResponse.newBuilder();
+      boolean closed = rand.nextBoolean();
+      if (closed) {
+        HRegionInfo hri = am.getRegionInfo(regionName);
+        sendTransitionReport(server, HRegionInfo.convert(hri), TransitionCode.CLOSED);
+      }
+      resp.setClosed(closed);
+      return resp.build();
+    }
+  }
+
+  private interface MockRSExecutor {
+    ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException;
+  }
+
+  private class MockRSProcedureDispatcher extends RSProcedureDispatcher {
+    private MockRSExecutor mockRsExec;
+
+    public MockRSProcedureDispatcher(final MasterServices master) {
+      super(master);
+    }
+
+    public void setMockRsExecutor(final MockRSExecutor mockRsExec) {
+      this.mockRsExec = mockRsExec;
+    }
+
+    @Override
+    protected void remoteDispatch(ServerName serverName, Set<RemoteProcedure> operations) {
+      submitTask(new MockRemoteCall(serverName, operations));
+    }
+
+    private class MockRemoteCall extends ExecuteProceduresRemoteCall {
+      public MockRemoteCall(final ServerName serverName,
+          final Set<RemoteProcedure> operations) {
+        super(serverName, operations);
+      }
+
+      @Override
+      protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
+          final ExecuteProceduresRequest request) throws IOException {
+        return mockRsExec.sendRequest(serverName, request);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
new file mode 100644
index 0000000..e4cec45
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
@@ -0,0 +1,185 @@
+/**
+ * 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.assignment;
+
+import java.io.IOException;
+
+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.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ServerName;
+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.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestAssignmentOnRSCrash {
+  private static final Log LOG = LogFactory.getLog(TestAssignmentOnRSCrash.class);
+
+  private static final TableName TEST_TABLE = TableName.valueOf("testb");
+  private static final String FAMILY_STR = "f";
+  private static final byte[] FAMILY = Bytes.toBytes(FAMILY_STR);
+  private static final int NUM_RS = 3;
+
+  private HBaseTestingUtility UTIL;
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    conf.set("hbase.balancer.tablesOnMaster", "none");
+  }
+
+  @Before
+  public void setup() throws Exception {
+    UTIL = new HBaseTestingUtility();
+
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(NUM_RS);
+
+    UTIL.createTable(TEST_TABLE, new byte[][] { FAMILY }, new byte[][] {
+      Bytes.toBytes("B"), Bytes.toBytes("D"), Bytes.toBytes("F"), Bytes.toBytes("L")
+    });
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test(timeout=30000)
+  public void testKillRsWithUserRegionWithData() throws Exception {
+    testCrashRsWithUserRegion(true, true);
+  }
+
+  @Test(timeout=30000)
+  public void testKillRsWithUserRegionWithoutData() throws Exception {
+    testCrashRsWithUserRegion(true, false);
+  }
+
+  @Test(timeout=30000)
+  public void testStopRsWithUserRegionWithData() throws Exception {
+    testCrashRsWithUserRegion(false, true);
+  }
+
+  @Test(timeout=30000)
+  public void testStopRsWithUserRegionWithoutData() throws Exception {
+    testCrashRsWithUserRegion(false, false);
+  }
+
+  private void testCrashRsWithUserRegion(final boolean kill, final boolean withData)
+      throws Exception {
+    final int NROWS = 100;
+    int nkilled = 0;
+    for (HRegionInfo hri: UTIL.getHBaseAdmin().getTableRegions(TEST_TABLE)) {
+      ServerName serverName = AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri);
+      if (AssignmentTestingUtil.isServerHoldingMeta(UTIL, serverName)) continue;
+
+      if (withData) {
+        testInsert(hri, NROWS);
+      }
+
+      // wait for regions to enter in transition and then to get out of transition
+      AssignmentTestingUtil.crashRs(UTIL, serverName, kill);
+      AssignmentTestingUtil.waitForRegionToBeInTransition(UTIL, hri);
+      UTIL.waitUntilNoRegionsInTransition();
+
+      if (withData) {
+        assertEquals(NROWS, testGet(hri, NROWS));
+      }
+
+      // region should be moved to another RS
+      assertNotEquals(serverName, AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri));
+
+      if (++nkilled == (NUM_RS - 1)) {
+        break;
+      }
+    }
+    assertTrue("expected RSs to be killed", nkilled > 0);
+  }
+
+  @Test(timeout=60000)
+  public void testKillRsWithMetaRegion() throws Exception {
+    testCrashRsWithMetaRegion(true);
+  }
+
+  @Test(timeout=60000)
+  public void testStopRsWithMetaRegion() throws Exception {
+    testCrashRsWithMetaRegion(false);
+  }
+
+  private void testCrashRsWithMetaRegion(final boolean kill) throws Exception {
+    int nkilled = 0;
+    for (HRegionInfo hri: AssignmentTestingUtil.getMetaRegions(UTIL)) {
+      ServerName serverName = AssignmentTestingUtil.crashRsWithRegion(UTIL, hri, kill);
+
+      // wait for region to enter in transition and then to get out of transition
+      AssignmentTestingUtil.waitForRegionToBeInTransition(UTIL, hri);
+      UTIL.waitUntilNoRegionsInTransition();
+      testGet(hri, 10);
+
+      // region should be moved to another RS
+      assertNotEquals(serverName, AssignmentTestingUtil.getServerHoldingRegion(UTIL, hri));
+
+      if (++nkilled == (NUM_RS - 1)) {
+        break;
+      }
+    }
+    assertTrue("expected RSs to be killed", nkilled > 0);
+  }
+
+  private void testInsert(final HRegionInfo hri, final int nrows) throws IOException {
+    final Table table = UTIL.getConnection().getTable(hri.getTable());
+    for (int i = 0; i < nrows; ++i) {
+      final byte[] row = Bytes.add(hri.getStartKey(), Bytes.toBytes(i));
+      final Put put = new Put(row);
+      put.addColumn(FAMILY, null, row);
+      table.put(put);
+    }
+  }
+
+  public int testGet(final HRegionInfo hri, final int nrows) throws IOException {
+    int nresults = 0;
+    final Table table = UTIL.getConnection().getTable(hri.getTable());
+    for (int i = 0; i < nrows; ++i) {
+      final byte[] row = Bytes.add(hri.getStartKey(), Bytes.toBytes(i));
+      final Result result = table.get(new Get(row));
+      if (result != null && !result.isEmpty() &&
+          Bytes.equals(row, result.getValue(FAMILY, null))) {
+        nresults++;
+      }
+    }
+    return nresults;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
new file mode 100644
index 0000000..51f28af
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
@@ -0,0 +1,239 @@
+/**
+ * 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.assignment;
+
+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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+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.client.Admin;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
+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.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestMergeTableRegionsProcedure {
+  private static final Log LOG = LogFactory.getLog(TestMergeTableRegionsProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static long nonceGroup = HConstants.NO_NONCE;
+  private static long nonce = HConstants.NO_NONCE;
+
+  private static final int initialRegionCount = 4;
+  private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
+  final static Configuration conf = UTIL.getConfiguration();
+  private static Admin admin;
+
+  private static void setupConf(Configuration conf) {
+    // Reduce the maximum attempts to speed up the test
+    conf.setInt("hbase.assignment.maximum.attempts", 3);
+    conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
+    conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
+
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(conf);
+    UTIL.startMiniCluster(1);
+    admin = UTIL.getHBaseAdmin();
+  }
+
+  @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 {
+    resetProcExecutorTestingKillFlag();
+    nonceGroup =
+        MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
+    nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
+    // Turn off balancer so it doesn't cut in and mess up our placements.
+    UTIL.getHBaseAdmin().setBalancerRunning(false, true);
+    // Turn off the meta scanner so it don't remove parent on us.
+    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
+    resetProcExecutorTestingKillFlag();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    resetProcExecutorTestingKillFlag();
+    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  private void resetProcExecutorTestingKillFlag() {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  /**
+   * This tests two region merges
+   */
+  @Test(timeout=60000)
+  public void testMergeTwoRegions() throws Exception {
+    final TableName tableName = TableName.valueOf("testMergeTwoRegions");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    List<HRegionInfo> tableRegions = createTable(tableName);
+
+    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+    regionsToMerge[0] = tableRegions.get(0);
+    regionsToMerge[1] = tableRegions.get(1);
+
+    long procId = procExec.submitProcedure(new MergeTableRegionsProcedure(
+      procExec.getEnvironment(), regionsToMerge, true));
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+    assertRegionCount(tableName, initialRegionCount - 1);
+  }
+
+  /**
+   * This tests two concurrent region merges
+   */
+  @Test(timeout=60000)
+  public void testMergeRegionsConcurrently() throws Exception {
+    final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    List<HRegionInfo> tableRegions = createTable(tableName);
+
+    HRegionInfo[] regionsToMerge1 = new HRegionInfo[2];
+    HRegionInfo[] regionsToMerge2 = new HRegionInfo[2];
+    regionsToMerge1[0] = tableRegions.get(0);
+    regionsToMerge1[1] = tableRegions.get(1);
+    regionsToMerge2[0] = tableRegions.get(2);
+    regionsToMerge2[1] = tableRegions.get(3);
+
+    long procId1 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+      procExec.getEnvironment(), regionsToMerge1, true));
+    long procId2 = procExec.submitProcedure(new MergeTableRegionsProcedure(
+      procExec.getEnvironment(), regionsToMerge2, true));
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+    assertRegionCount(tableName, initialRegionCount - 2);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    List<HRegionInfo> tableRegions = createTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+    regionsToMerge[0] = tableRegions.get(0);
+    regionsToMerge[1] = tableRegions.get(1);
+
+    long procId = procExec.submitProcedure(
+      new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
+
+    // Restart the executor and execute the step twice
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+
+    assertRegionCount(tableName, initialRegionCount - 1);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    List<HRegionInfo> tableRegions = createTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    HRegionInfo[] regionsToMerge = new HRegionInfo[2];
+    regionsToMerge[0] = tableRegions.get(0);
+    regionsToMerge[1] = tableRegions.get(1);
+
+    long procId = procExec.submitProcedure(
+      new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true));
+
+    // Failing before MERGE_TABLE_REGIONS_UPDATE_META we should trigger the rollback
+    // NOTE: the 5 (number before MERGE_TABLE_REGIONS_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);
+  }
+
+  private List<HRegionInfo> createTable(final TableName tableName)
+      throws Exception {
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor(FAMILY));
+    byte[][] splitRows = new byte[initialRegionCount - 1][];
+    for (int i = 0; i < splitRows.length; ++i) {
+      splitRows[i] = Bytes.toBytes(String.format("%d", i));
+    }
+    admin.createTable(desc, splitRows);
+    return assertRegionCount(tableName, initialRegionCount);
+  }
+
+  public List<HRegionInfo> assertRegionCount(final TableName tableName, final int nregions)
+      throws Exception {
+    UTIL.waitUntilNoRegionsInTransition();
+    List<HRegionInfo> tableRegions = admin.getTableRegions(tableName);
+    assertEquals(nregions, tableRegions.size());
+    return tableRegions;
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}
\ 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/assignment/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
new file mode 100644
index 0000000..1d2481d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
@@ -0,0 +1,226 @@
+/**
+ * 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.assignment;
+
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestRegionStates {
+  private static final Log LOG = LogFactory.getLog(TestRegionStates.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static ThreadPoolExecutor threadPool;
+  private static ExecutorCompletionService executorService;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    threadPool = Threads.getBoundedCachedThreadPool(32, 60L, TimeUnit.SECONDS,
+      Threads.newDaemonThreadFactory("ProcedureDispatcher",
+        new UncaughtExceptionHandler() {
+          @Override
+          public void uncaughtException(Thread t, Throwable e) {
+            LOG.warn("Failed thread " + t.getName(), e);
+          }
+        }));
+    executorService = new ExecutorCompletionService(threadPool);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    threadPool.shutdown();
+  }
+
+  @Before
+  public void testSetup() {
+  }
+
+  @After
+  public void testTearDown() throws Exception {
+    while (true) {
+      Future<Object> f = executorService.poll();
+      if (f == null) break;
+      f.get();
+    }
+  }
+
+  private static void waitExecutorService(final int count) throws Exception {
+    for (int i = 0; i < count; ++i) {
+      executorService.take().get();
+    }
+  }
+
+  // ==========================================================================
+  //  Regions related
+  // ==========================================================================
+
+  @Test
+  public void testRegionDoubleCreation() throws Exception {
+    // NOTE: HRegionInfo sort by table first, so we are relying on that
+    final TableName TABLE_NAME_A = TableName.valueOf("testOrderedByTableA");
+    final TableName TABLE_NAME_B = TableName.valueOf("testOrderedByTableB");
+    final TableName TABLE_NAME_C = TableName.valueOf("testOrderedByTableC");
+    final RegionStates stateMap = new RegionStates();
+    final int NRUNS = 1000;
+    final int NSMALL_RUNS = 3;
+
+    // add some regions for table B
+    for (int i = 0; i < NRUNS; ++i) {
+      addRegionNode(stateMap, TABLE_NAME_B, i);
+    }
+    // re-add the regions for table B
+    for (int i = 0; i < NRUNS; ++i) {
+      addRegionNode(stateMap, TABLE_NAME_B, i);
+    }
+    waitExecutorService(NRUNS * 2);
+
+    // add two other tables A and C that will be placed before and after table B (sort order)
+    for (int i = 0; i < NSMALL_RUNS; ++i) {
+      addRegionNode(stateMap, TABLE_NAME_A, i);
+      addRegionNode(stateMap, TABLE_NAME_C, i);
+    }
+
+    // check for the list of regions of the 3 tables
+    checkTableRegions(stateMap, TABLE_NAME_A, NSMALL_RUNS);
+    checkTableRegions(stateMap, TABLE_NAME_B, NRUNS);
+    checkTableRegions(stateMap, TABLE_NAME_C, NSMALL_RUNS);
+  }
+
+  private void checkTableRegions(final RegionStates stateMap,
+      final TableName tableName, final int nregions) {
+    List<HRegionInfo> hris = stateMap.getRegionsOfTable(tableName);
+    assertEquals(nregions, hris.size());
+    for (int i = 1; i < hris.size(); ++i) {
+      long a = Bytes.toLong(hris.get(i - 1).getStartKey());
+      long b = Bytes.toLong(hris.get(i + 0).getStartKey());
+      assertEquals(b, a + 1);
+    }
+  }
+
+  private void addRegionNode(final RegionStates stateMap,
+      final TableName tableName, final long regionId) {
+    executorService.submit(new Callable<Object>() {
+      @Override
+      public Object call() {
+        HRegionInfo hri = new HRegionInfo(tableName,
+          Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
+        return stateMap.getOrCreateRegionNode(hri);
+      }
+    });
+  }
+
+  private Object createRegionNode(final RegionStates stateMap,
+      final TableName tableName, final long regionId) {
+    return stateMap.getOrCreateRegionNode(createRegionInfo(tableName, regionId));
+  }
+
+  private HRegionInfo createRegionInfo(final TableName tableName, final long regionId) {
+    return new HRegionInfo(tableName,
+      Bytes.toBytes(regionId), Bytes.toBytes(regionId + 1), false, 0);
+  }
+
+  @Test
+  public void testPerf() throws Exception {
+    final TableName TABLE_NAME = TableName.valueOf("testPerf");
+    final int NRUNS = 1000000; // 1M
+    final RegionStates stateMap = new RegionStates();
+
+    long st = System.currentTimeMillis();
+    for (int i = 0; i < NRUNS; ++i) {
+      final int regionId = i;
+      executorService.submit(new Callable<Object>() {
+        @Override
+        public Object call() {
+          HRegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
+          return stateMap.getOrCreateRegionNode(hri);
+        }
+      });
+    }
+    waitExecutorService(NRUNS);
+    long et = System.currentTimeMillis();
+    LOG.info(String.format("PERF STATEMAP INSERT: %s %s/sec",
+      StringUtils.humanTimeDiff(et - st),
+      StringUtils.humanSize(NRUNS / ((et - st) / 1000.0f))));
+
+    st = System.currentTimeMillis();
+    for (int i = 0; i < NRUNS; ++i) {
+      final int regionId = i;
+      executorService.submit(new Callable<Object>() {
+        @Override
+        public Object call() {
+          HRegionInfo hri = createRegionInfo(TABLE_NAME, regionId);
+          return stateMap.getRegionState(hri);
+        }
+      });
+    }
+
+    waitExecutorService(NRUNS);
+    et = System.currentTimeMillis();
+    LOG.info(String.format("PERF STATEMAP GET: %s %s/sec",
+      StringUtils.humanTimeDiff(et - st),
+      StringUtils.humanSize(NRUNS / ((et - st) / 1000.0f))));
+  }
+
+  @Test
+  public void testPerfSingleThread() {
+    final TableName TABLE_NAME = TableName.valueOf("testPerf");
+    final int NRUNS = 1 * 1000000; // 1M
+
+    final RegionStates stateMap = new RegionStates();
+    long st = System.currentTimeMillis();
+    for (int i = 0; i < NRUNS; ++i) {
+      stateMap.createRegionNode(createRegionInfo(TABLE_NAME, i));
+    }
+    long et = System.currentTimeMillis();
+    LOG.info(String.format("PERF SingleThread: %s %s/sec",
+        StringUtils.humanTimeDiff(et - st),
+      StringUtils.humanSize(NRUNS / ((et - st) / 1000.0f))));
+  }
+
+  // ==========================================================================
+  //  Server related
+  // ==========================================================================
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
new file mode 100644
index 0000000..7af9d67
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
@@ -0,0 +1,427 @@
+/**
+ * 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.assignment;
+
+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.CategoryBasedTimeout;
+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.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
+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.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;
+import org.junit.rules.TestRule;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestSplitTableRegionProcedure {
+  private static final Log LOG = LogFactory.getLog(TestSplitTableRegionProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
+      withLookingForStuckThread(true).build();
+
+  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()) {
+      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 3 (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 = 3;
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
+    // check that we have only 1 region
+    assertEquals(1, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
+    assertEquals(1, daughters.size());
+    verifyData(daughters.get(0), startRowNum, rowCount,
+    Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes(ColumnFamilyName2));
+  }
+
+  @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
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    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/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index ce8b0c6..99dcf1d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.List;
 import java.util.TreeSet;
+import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -48,7 +49,11 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterMetaBootstrap;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -59,7 +64,45 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 public class MasterProcedureTestingUtility {
   private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
 
-  private MasterProcedureTestingUtility() {
+  private MasterProcedureTestingUtility() { }
+
+  public static void restartMasterProcedureExecutor(ProcedureExecutor<MasterProcedureEnv> procExec)
+      throws Exception {
+    final MasterProcedureEnv env = procExec.getEnvironment();
+    final HMaster master = (HMaster)env.getMasterServices();
+    ProcedureTestingUtility.restart(procExec, true, true,
+      // stop services
+      new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          final AssignmentManager am = env.getAssignmentManager();
+          // try to simulate a master restart by removing the ServerManager states about seqIDs
+          for (RegionState regionState: am.getRegionStates().getRegionStates()) {
+            env.getMasterServices().getServerManager().removeRegion(regionState.getRegion());
+          }
+          am.stop();
+          master.setInitialized(false);
+          return null;
+        }
+      },
+      // restart services
+      new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          final AssignmentManager am = env.getAssignmentManager();
+          am.start();
+          if (true) {
+            MasterMetaBootstrap metaBootstrap = new MasterMetaBootstrap(master,
+                TaskMonitor.get().createStatus("meta"));
+            metaBootstrap.splitMetaLogsBeforeAssignment();
+            metaBootstrap.assignMeta();
+            metaBootstrap.processDeadServers();
+          }
+          am.joinCluster();
+          master.setInitialized(true);
+          return null;
+        }
+      });
   }
 
   // ==========================================================================
@@ -295,6 +338,9 @@ public class MasterProcedureTestingUtility {
     return put;
   }
 
+  // ==========================================================================
+  //  Procedure Helpers
+  // ==========================================================================
   public static long generateNonceGroup(final HMaster master) {
     return master.getClusterConnection().getNonceGenerator().getNonceGroup();
   }
@@ -318,13 +364,6 @@ public class MasterProcedureTestingUtility {
    * finish.
    * @see #testRecoveryAndDoubleExecution(ProcedureExecutor, long)
    */
-  public static void testRecoveryAndDoubleExecution(
-      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
-      final int numSteps) throws Exception {
-    testRecoveryAndDoubleExecution(procExec, procId, numSteps, true);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-  }
-
   private static void testRecoveryAndDoubleExecution(
       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
       final int numSteps, final boolean expectExecRunning) throws Exception {
@@ -336,9 +375,9 @@ public class MasterProcedureTestingUtility {
     //   restart executor/store
     //   execute step N - save on store
     for (int i = 0; i < numSteps; ++i) {
-      LOG.info("Restart " + i + " exec state: " + procExec.getProcedure(procId));
+      LOG.info("Restart " + i + " exec state=" + procExec.getProcedure(procId));
       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
-      ProcedureTestingUtility.restart(procExec);
+      restartMasterProcedureExecutor(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
 
@@ -366,8 +405,8 @@ public class MasterProcedureTestingUtility {
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     assertEquals(false, procExec.isRunning());
     for (int i = 0; !procExec.isFinished(procId); ++i) {
-      LOG.info("Restart " + i + " exec state: " + procExec.getProcedure(procId));
-      ProcedureTestingUtility.restart(procExec);
+      LOG.info("Restart " + i + " exec state=" + procExec.getProcedure(procId));
+      restartMasterProcedureExecutor(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
     assertEquals(true, procExec.isRunning());
@@ -399,7 +438,7 @@ public class MasterProcedureTestingUtility {
       for (int i = 0; !procExec.isFinished(procId); ++i) {
         LOG.info("Restart " + i + " rollback state: " + procExec.getProcedure(procId));
         ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
-        ProcedureTestingUtility.restart(procExec);
+        restartMasterProcedureExecutor(procExec);
         ProcedureTestingUtility.waitProcedure(procExec, procId);
       }
     } finally {
@@ -444,7 +483,7 @@ public class MasterProcedureTestingUtility {
     try {
       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
       LOG.info("Restart and rollback procId=" + procId);
-      ProcedureTestingUtility.restart(procExec);
+      restartMasterProcedureExecutor(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     } finally {
       assertTrue(procExec.unregisterListener(abortListener));


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
deleted file mode 100644
index 3a2a6d7..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
+++ /dev/null
@@ -1,268 +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;
-
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-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.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.RegionLocations;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.MultiHConnection;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * A helper to persist region state in meta. We may change this class
- * to StateStore later if we also use it to store other states in meta
- */
-@InterfaceAudience.Private
-public class RegionStateStore {
-  private static final Log LOG = LogFactory.getLog(RegionStateStore.class);
-
-  /** The delimiter for meta columns for replicaIds &gt; 0 */
-  protected static final char META_REPLICA_ID_DELIMITER = '_';
-
-  private volatile Region metaRegion;
-  private volatile boolean initialized;
-  private MultiHConnection multiHConnection;
-  private final MasterServices server;
-
-  /**
-   * Returns the {@link ServerName} from catalog table {@link Result}
-   * where the region is transitioning. It should be the same as
-   * {@link MetaTableAccessor#getServerName(Result,int)} if the server is at OPEN state.
-   * @param r Result to pull the transitioning server name from
-   * @return A ServerName instance or {@link MetaTableAccessor#getServerName(Result,int)}
-   * if necessary fields not found or empty.
-   */
-  static ServerName getRegionServer(final Result r, int replicaId) {
-    Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId));
-    if (cell == null || cell.getValueLength() == 0) {
-      RegionLocations locations = MetaTableAccessor.getRegionLocations(r);
-      if (locations != null) {
-        HRegionLocation location = locations.getRegionLocation(replicaId);
-        if (location != null) {
-          return location.getServerName();
-        }
-      }
-      return null;
-    }
-    return ServerName.parseServerName(Bytes.toString(cell.getValueArray(),
-      cell.getValueOffset(), cell.getValueLength()));
-  }
-
-  private static byte[] getServerNameColumn(int replicaId) {
-    return replicaId == 0
-        ? HConstants.SERVERNAME_QUALIFIER
-        : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-          + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
-  }
-
-  /**
-   * Pull the region state from a catalog table {@link Result}.
-   * @param r Result to pull the region state from
-   * @return the region state, or OPEN if there's no value written.
-   */
-  static State getRegionState(final Result r, int replicaId) {
-    Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(replicaId));
-    if (cell == null || cell.getValueLength() == 0) return State.OPEN;
-    return State.valueOf(Bytes.toString(cell.getValueArray(),
-      cell.getValueOffset(), cell.getValueLength()));
-  }
-
-  private static byte[] getStateColumn(int replicaId) {
-    return replicaId == 0
-        ? HConstants.STATE_QUALIFIER
-        : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
-          + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
-  }
-
-  /**
-   * Check if we should persist a state change in meta. Generally it's
-   * better to persist all state changes. However, we should not do that
-   * if the region is not in meta at all. Based on the state and the
-   * previous state, we can identify if a user region has an entry
-   * in meta. For example, merged regions are deleted from meta;
-   * New merging parents, or splitting daughters are
-   * not created in meta yet.
-   */
-  private boolean shouldPersistStateChange(
-      HRegionInfo hri, RegionState state, RegionState oldState) {
-    return !hri.isMetaRegion() && !RegionStates.isOneOfStates(
-      state, State.MERGING_NEW, State.SPLITTING_NEW, State.MERGED)
-      && !(RegionStates.isOneOfStates(state, State.OFFLINE)
-        && RegionStates.isOneOfStates(oldState, State.MERGING_NEW,
-          State.SPLITTING_NEW, State.MERGED));
-  }
-
-  RegionStateStore(final MasterServices server) {
-    this.server = server;
-    initialized = false;
-  }
-
-  void start() throws IOException {
-    if (server instanceof RegionServerServices) {
-      metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
-        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
-    }
-    // When meta is not colocated on master
-    if (metaRegion == null) {
-      Configuration conf = server.getConfiguration();
-      // Config to determine the no of HConnections to META.
-      // A single Connection should be sufficient in most cases. Only if
-      // you are doing lot of writes (>1M) to META,
-      // increasing this value might improve the write throughput.
-      multiHConnection =
-          new MultiHConnection(conf, conf.getInt("hbase.regionstatestore.meta.connection", 1));
-    }
-    initialized = true;
-  }
-
-  void stop() {
-    initialized = false;
-    if (multiHConnection != null) {
-      multiHConnection.close();
-    }
-  }
-
-  void updateRegionState(long openSeqNum,
-      RegionState newState, RegionState oldState) {
-    try {
-      HRegionInfo hri = newState.getRegion();
-
-      // Update meta before checking for initialization. Meta state stored in zk.
-      if (hri.isMetaRegion()) {
-        // persist meta state in MetaTableLocator (which in turn is zk storage currently)
-        try {
-          MetaTableLocator.setMetaLocation(server.getZooKeeper(),
-            newState.getServerName(), hri.getReplicaId(), newState.getState());
-          return; // Done
-        } catch (KeeperException e) {
-          throw new IOException("Failed to update meta ZNode", e);
-        }
-      }
-
-      if (!initialized
-          || !shouldPersistStateChange(hri, newState, oldState)) {
-        return;
-      }
-
-      ServerName oldServer = oldState != null ? oldState.getServerName() : null;
-      ServerName serverName = newState.getServerName();
-      State state = newState.getState();
-
-      int replicaId = hri.getReplicaId();
-      Put metaPut = new Put(MetaTableAccessor.getMetaKeyForRegion(hri));
-      StringBuilder info = new StringBuilder("Updating hbase:meta row ");
-      info.append(hri.getRegionNameAsString()).append(" with state=").append(state);
-      if (serverName != null && !serverName.equals(oldServer)) {
-        metaPut.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
-          Bytes.toBytes(serverName.getServerName()));
-        info.append(", sn=").append(serverName);
-      }
-      if (openSeqNum >= 0) {
-        Preconditions.checkArgument(state == State.OPEN
-          && serverName != null, "Open region should be on a server");
-        MetaTableAccessor.addLocation(metaPut, serverName, openSeqNum, -1, replicaId);
-        info.append(", openSeqNum=").append(openSeqNum);
-        info.append(", server=").append(serverName);
-      }
-      metaPut.addImmutable(HConstants.CATALOG_FAMILY, getStateColumn(replicaId),
-        Bytes.toBytes(state.name()));
-      LOG.info(info);
-      HTableDescriptor descriptor = server.getTableDescriptors().get(hri.getTable());
-      boolean serial = false;
-      if (descriptor != null) {
-        serial = server.getTableDescriptors().get(hri.getTable()).hasSerialReplicationScope();
-      }
-      boolean shouldPutBarrier = serial && state == State.OPEN;
-      // Persist the state change to meta
-      if (metaRegion != null) {
-        try {
-          // Assume meta is pinned to master.
-          // At least, that's what we want.
-          metaRegion.put(metaPut);
-          if (shouldPutBarrier) {
-            Put barrierPut = MetaTableAccessor.makeBarrierPut(hri.getEncodedNameAsBytes(),
-                openSeqNum, hri.getTable().getName());
-            metaRegion.put(barrierPut);
-          }
-          return; // Done here
-        } catch (Throwable t) {
-          // In unit tests, meta could be moved away by intention
-          // So, the shortcut is gone. We won't try to establish the
-          // shortcut any more because we prefer meta to be pinned
-          // to the master
-          synchronized (this) {
-            if (metaRegion != null) {
-              LOG.info("Meta region shortcut failed", t);
-              if (multiHConnection == null) {
-                multiHConnection = new MultiHConnection(server.getConfiguration(), 1);
-              }
-              metaRegion = null;
-            }
-          }
-        }
-      }
-      // Called when meta is not on master
-      List<Put> list = shouldPutBarrier ?
-          Arrays.asList(metaPut, MetaTableAccessor.makeBarrierPut(hri.getEncodedNameAsBytes(),
-              openSeqNum, hri.getTable().getName())) : Collections.singletonList(metaPut);
-      multiHConnection.processBatchCallback(list, TableName.META_TABLE_NAME, null, null);
-
-    } catch (IOException ioe) {
-      LOG.error("Failed to persist region state " + newState, ioe);
-      server.abort("Failed to update region location", ioe);
-    }
-  }
-
-  void splitRegion(HRegionInfo p,
-      HRegionInfo a, HRegionInfo b, ServerName sn, int regionReplication) throws IOException {
-    MetaTableAccessor.splitRegion(server.getConnection(), p, a, b, sn, regionReplication,
-        server.getTableDescriptors().get(p.getTable()).hasSerialReplicationScope());
-  }
-
-  void mergeRegions(HRegionInfo p,
-      HRegionInfo a, HRegionInfo b, ServerName sn, int regionReplication) throws IOException {
-    MetaTableAccessor.mergeRegions(server.getConnection(), p, a, b, sn, regionReplication,
-        EnvironmentEdgeManager.currentTime(),
-        server.getTableDescriptors().get(p.getTable()).hasSerialReplicationScope());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
deleted file mode 100644
index a1e24f2..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
+++ /dev/null
@@ -1,1170 +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;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * Region state accountant. It holds the states of all regions in the memory.
- * In normal scenario, it should match the meta table and the true region states.
- *
- * This map is used by AssignmentManager to track region states.
- */
-@InterfaceAudience.Private
-public class RegionStates {
-  private static final Log LOG = LogFactory.getLog(RegionStates.class);
-
-  public final static RegionStateStampComparator REGION_STATE_COMPARATOR =
-    new RegionStateStampComparator();
-
-  // This comparator sorts the RegionStates by time stamp then Region name.
-  // Comparing by timestamp alone can lead us to discard different RegionStates that happen
-  // to share a timestamp.
-  private static class RegionStateStampComparator implements Comparator<RegionState> {
-    @Override
-    public int compare(RegionState l, RegionState r) {
-      return Long.compare(l.getStamp(), r.getStamp()) == 0 ?
-          Bytes.compareTo(l.getRegion().getRegionName(), r.getRegion().getRegionName()) :
-          Long.compare(l.getStamp(), r.getStamp());
-    }
-  }
-
-  /**
-   * Regions currently in transition.
-   */
-  final HashMap<String, RegionState> regionsInTransition = new HashMap<>();
-
-  /**
-   * Region encoded name to state map.
-   * All the regions should be in this map.
-   */
-  private final Map<String, RegionState> regionStates = new HashMap<>();
-
-  /**
-   * Holds mapping of table -> region state
-   */
-  private final Map<TableName, Map<String, RegionState>> regionStatesTableIndex = new HashMap<>();
-
-  /**
-   * Server to regions assignment map.
-   * Contains the set of regions currently assigned to a given server.
-   */
-  private final Map<ServerName, Set<HRegionInfo>> serverHoldings = new HashMap<>();
-
-  /**
-   * Maintains the mapping from the default region to the replica regions.
-   */
-  private final Map<HRegionInfo, Set<HRegionInfo>> defaultReplicaToOtherReplicas = new HashMap<>();
-
-  /**
-   * Region to server assignment map.
-   * Contains the server a given region is currently assigned to.
-   */
-  private final TreeMap<HRegionInfo, ServerName> regionAssignments = new TreeMap<>();
-
-  /**
-   * Encoded region name to server assignment map for re-assignment
-   * purpose. Contains the server a given region is last known assigned
-   * to, which has not completed log splitting, so not assignable.
-   * If a region is currently assigned, this server info in this
-   * map should be the same as that in regionAssignments.
-   * However the info in regionAssignments is cleared when the region
-   * is offline while the info in lastAssignments is cleared when
-   * the region is closed or the server is dead and processed.
-   */
-  private final HashMap<String, ServerName> lastAssignments = new HashMap<>();
-
-  /**
-   * Encoded region name to server assignment map for the
-   * purpose to clean up serverHoldings when a region is online
-   * on a new server. When the region is offline from the previous
-   * server, we cleaned up regionAssignments so that it has the
-   * latest assignment map. But we didn't clean up serverHoldings
-   * to match the meta. We need this map to find out the old server
-   * whose serverHoldings needs cleanup, given a moved region.
-   */
-  private final HashMap<String, ServerName> oldAssignments = new HashMap<>();
-
-  /**
-   * Map a host port pair string to the latest start code
-   * of a region server which is known to be dead. It is dead
-   * to us, but server manager may not know it yet.
-   */
-  private final HashMap<String, Long> deadServers = new HashMap<>();
-
-  /**
-   * Map a dead servers to the time when log split is done.
-   * Since log splitting is not ordered, we have to remember
-   * all processed instances. The map is cleaned up based
-   * on a configured time. By default, we assume a dead
-   * server should be done with log splitting in two hours.
-   */
-  private final HashMap<ServerName, Long> processedServers = new HashMap<>();
-  private long lastProcessedServerCleanTime;
-
-  private final TableStateManager tableStateManager;
-  private final RegionStateStore regionStateStore;
-  private final ServerManager serverManager;
-  private final MasterServices server;
-
-  // The maximum time to keep a log split info in region states map
-  static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime";
-  static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours
-
-  RegionStates(final MasterServices master, final TableStateManager tableStateManager,
-      final ServerManager serverManager, final RegionStateStore regionStateStore) {
-    this.tableStateManager = tableStateManager;
-    this.regionStateStore = regionStateStore;
-    this.serverManager = serverManager;
-    this.server = master;
-  }
-
-  /**
-   * @return a copy of the region assignment map
-   */
-  public synchronized Map<HRegionInfo, ServerName> getRegionAssignments() {
-    return new TreeMap<>(regionAssignments);
-  }
-
-  /**
-   * Return the replicas (including default) for the regions grouped by ServerName
-   * @param regions
-   * @return a pair containing the groupings as a map
-   */
-  synchronized Map<ServerName, List<HRegionInfo>> getRegionAssignments(
-    Collection<HRegionInfo> regions) {
-    Map<ServerName, List<HRegionInfo>> map = new HashMap<>();
-    for (HRegionInfo region : regions) {
-      HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(region);
-      Set<HRegionInfo> allReplicas = defaultReplicaToOtherReplicas.get(defaultReplica);
-      if (allReplicas != null) {
-        for (HRegionInfo hri : allReplicas) {
-          ServerName server = regionAssignments.get(hri);
-          if (server != null) {
-            List<HRegionInfo> regionsOnServer = map.get(server);
-            if (regionsOnServer == null) {
-              regionsOnServer = new ArrayList<>(1);
-              map.put(server, regionsOnServer);
-            }
-            regionsOnServer.add(hri);
-          }
-        }
-      }
-    }
-    return map;
-  }
-
-  public synchronized ServerName getRegionServerOfRegion(HRegionInfo hri) {
-    return regionAssignments.get(hri);
-  }
-
-  /**
-   * Get regions in transition and their states
-   */
-  public synchronized Set<RegionState> getRegionsInTransition() {
-    return new HashSet<>(regionsInTransition.values());
-  }
-
-  public synchronized SortedSet<RegionState> getRegionsInTransitionOrderedByTimestamp() {
-    final TreeSet<RegionState> rit = new TreeSet<>(REGION_STATE_COMPARATOR);
-    for (RegionState rs: regionsInTransition.values()) {
-      rit.add(rs);
-    }
-    return rit;
-  }
-
-  /**
-   * Get the number of regions in transition.
-   */
-  public synchronized int getRegionsInTransitionCount() {
-    return regionsInTransition.size();
-  }
-
-  /**
-   * @return True if specified region in transition.
-   */
-  public synchronized boolean isRegionInTransition(final HRegionInfo hri) {
-    return regionsInTransition.containsKey(hri.getEncodedName());
-  }
-
-  /**
-   * @return True if specified region in transition.
-   */
-  public synchronized boolean isRegionInTransition(final String encodedName) {
-    return regionsInTransition.containsKey(encodedName);
-  }
-
-  /**
-   * @return True if any region in transition.
-   */
-  public synchronized boolean isRegionsInTransition() {
-    return !regionsInTransition.isEmpty();
-  }
-
-  /**
-   * @return True if hbase:meta table region is in transition.
-   */
-  public synchronized boolean isMetaRegionInTransition() {
-    for (RegionState state : regionsInTransition.values()) {
-      if (state.getRegion().isMetaRegion()) return true;
-    }
-    return false;
-  }
-
-  /**
-   * @return True if specified region assigned, and not in transition.
-   */
-  public synchronized boolean isRegionOnline(final HRegionInfo hri) {
-    return !isRegionInTransition(hri) && regionAssignments.containsKey(hri);
-  }
-
-  /**
-   * @return True if specified region offline/closed, but not in transition.
-   * If the region is not in the map, it is offline to us too.
-   */
-  public synchronized boolean isRegionOffline(final HRegionInfo hri) {
-    return getRegionState(hri) == null || (!isRegionInTransition(hri)
-      && isRegionInState(hri, State.OFFLINE, State.CLOSED));
-  }
-
-  /**
-   * @return True if specified region is in one of the specified states.
-   */
-  public boolean isRegionInState(
-      final HRegionInfo hri, final State... states) {
-    return isRegionInState(hri.getEncodedName(), states);
-  }
-
-  /**
-   * @return True if specified region is in one of the specified states.
-   */
-  public boolean isRegionInState(
-      final String encodedName, final State... states) {
-    RegionState regionState = getRegionState(encodedName);
-    return isOneOfStates(regionState, states);
-  }
-
-  /**
-   * Wait for the state map to be updated by assignment manager.
-   */
-  public synchronized void waitForUpdate(
-      final long timeout) throws InterruptedException {
-    this.wait(timeout);
-  }
-
-  /**
-   * Get region transition state
-   */
-  public RegionState getRegionTransitionState(final HRegionInfo hri) {
-    return getRegionTransitionState(hri.getEncodedName());
-  }
-
-  /**
-   * Get region transition state
-   */
-  public synchronized RegionState
-      getRegionTransitionState(final String encodedName) {
-    return regionsInTransition.get(encodedName);
-  }
-
-  /**
-   * Add a list of regions to RegionStates. If a region is split
-   * and offline, its state will be SPLIT. Otherwise, its state will
-   * be OFFLINE. Region already in RegionStates will be skipped.
-   */
-  public void createRegionStates(
-      final List<HRegionInfo> hris) {
-    for (HRegionInfo hri: hris) {
-      createRegionState(hri);
-    }
-  }
-
-  /**
-   * Add a region to RegionStates. If the region is split
-   * and offline, its state will be SPLIT. Otherwise, its state will
-   * be OFFLINE. If it is already in RegionStates, this call has
-   * no effect, and the original state is returned.
-   */
-  public RegionState createRegionState(final HRegionInfo hri) {
-    return createRegionState(hri, null, null, null);
-  }
-
-  /**
-   * Add a region to RegionStates with the specified state.
-   * If the region is already in RegionStates, this call has
-   * no effect, and the original state is returned.
-   *
-   * @param hri the region info to create a state for
-   * @param newState the state to the region in set to
-   * @param serverName the server the region is transitioning on
-   * @param lastHost the last server that hosts the region
-   * @return the current state
-   */
-  public synchronized RegionState createRegionState(final HRegionInfo hri,
-      State newState, ServerName serverName, ServerName lastHost) {
-    if (newState == null || (newState == State.OPEN && serverName == null)) {
-      newState =  State.OFFLINE;
-    }
-    if (hri.isOffline() && hri.isSplit()) {
-      newState = State.SPLIT;
-      serverName = null;
-    }
-    String encodedName = hri.getEncodedName();
-    RegionState regionState = regionStates.get(encodedName);
-    if (regionState != null) {
-      LOG.warn("Tried to create a state for a region already in RegionStates, "
-        + "used existing: " + regionState + ", ignored new: " + newState);
-    } else {
-      regionState = new RegionState(hri, newState, serverName);
-      putRegionState(regionState);
-      if (newState == State.OPEN) {
-        if (!serverName.equals(lastHost)) {
-          LOG.warn("Open region's last host " + lastHost
-            + " should be the same as the current one " + serverName
-            + ", ignored the last and used the current one");
-          lastHost = serverName;
-        }
-        lastAssignments.put(encodedName, lastHost);
-        regionAssignments.put(hri, lastHost);
-      } else if (!isOneOfStates(regionState, State.MERGED, State.SPLIT, State.OFFLINE)) {
-        regionsInTransition.put(encodedName, regionState);
-      }
-      if (lastHost != null && newState != State.SPLIT) {
-        addToServerHoldings(lastHost, hri);
-        if (newState != State.OPEN) {
-          oldAssignments.put(encodedName, lastHost);
-        }
-      }
-    }
-    return regionState;
-  }
-
-  private RegionState putRegionState(RegionState regionState) {
-    HRegionInfo hri = regionState.getRegion();
-    String encodedName = hri.getEncodedName();
-    TableName table = hri.getTable();
-    RegionState oldState = regionStates.put(encodedName, regionState);
-    Map<String, RegionState> map = regionStatesTableIndex.get(table);
-    if (map == null) {
-      map = new HashMap<>();
-      regionStatesTableIndex.put(table, map);
-    }
-    map.put(encodedName, regionState);
-    return oldState;
-  }
-
-  /**
-   * Update a region state. It will be put in transition if not already there.
-   */
-  public RegionState updateRegionState(
-      final HRegionInfo hri, final State state) {
-    RegionState regionState = getRegionState(hri.getEncodedName());
-    return updateRegionState(hri, state,
-      regionState == null ? null : regionState.getServerName());
-  }
-
-  /**
-   * Update a region state. It will be put in transition if not already there.
-   */
-  public RegionState updateRegionState(
-      final HRegionInfo hri, final State state, final ServerName serverName) {
-    return updateRegionState(hri, state, serverName, HConstants.NO_SEQNUM);
-  }
-
-  public void regionOnline(final HRegionInfo hri, final ServerName serverName) {
-    regionOnline(hri, serverName, HConstants.NO_SEQNUM);
-  }
-
-  /**
-   * A region is online, won't be in transition any more.
-   * We can't confirm it is really online on specified region server
-   * because it hasn't been put in region server's online region list yet.
-   */
-  public void regionOnline(final HRegionInfo hri, final ServerName serverName, long openSeqNum) {
-    String encodedName = hri.getEncodedName();
-    if (!serverManager.isServerOnline(serverName)) {
-      // This is possible if the region server dies before master gets a
-      // chance to handle ZK event in time. At this time, if the dead server
-      // is already processed by SSH, we should ignore this event.
-      // If not processed yet, ignore and let SSH deal with it.
-      LOG.warn("Ignored, " + encodedName + " was opened on a dead server: " + serverName);
-      return;
-    }
-    updateRegionState(hri, State.OPEN, serverName, openSeqNum);
-
-    synchronized (this) {
-      RegionState regionState = regionsInTransition.remove(encodedName);
-      // When region is online and remove from regionsInTransition,
-      // update the RIT duration to assignment manager metrics
-      if (regionState != null && this.server.getAssignmentManager() != null) {
-        long ritDuration = System.currentTimeMillis() - regionState.getStamp()
-            + regionState.getRitDuration();
-        this.server.getAssignmentManager().getAssignmentManagerMetrics()
-            .updateRitDuration(ritDuration);
-      }
-      ServerName oldServerName = regionAssignments.put(hri, serverName);
-      if (!serverName.equals(oldServerName)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName);
-        }
-        addToServerHoldings(serverName, hri);
-        addToReplicaMapping(hri);
-        if (oldServerName == null) {
-          oldServerName = oldAssignments.remove(encodedName);
-        }
-        if (oldServerName != null
-            && !oldServerName.equals(serverName)
-            && serverHoldings.containsKey(oldServerName)) {
-          LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
-          removeFromServerHoldings(oldServerName, hri);
-        }
-      }
-    }
-  }
-
-  private void addToServerHoldings(ServerName serverName, HRegionInfo hri) {
-    Set<HRegionInfo> regions = serverHoldings.get(serverName);
-    if (regions == null) {
-      regions = new HashSet<>();
-      serverHoldings.put(serverName, regions);
-    }
-    regions.add(hri);
-  }
-
-  private void addToReplicaMapping(HRegionInfo hri) {
-    HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
-    Set<HRegionInfo> replicas =
-        defaultReplicaToOtherReplicas.get(defaultReplica);
-    if (replicas == null) {
-      replicas = new HashSet<>();
-      defaultReplicaToOtherReplicas.put(defaultReplica, replicas);
-    }
-    replicas.add(hri);
-  }
-
-  private void removeFromServerHoldings(ServerName serverName, HRegionInfo hri) {
-    Set<HRegionInfo> oldRegions = serverHoldings.get(serverName);
-    oldRegions.remove(hri);
-    if (oldRegions.isEmpty()) {
-      serverHoldings.remove(serverName);
-    }
-  }
-
-  private void removeFromReplicaMapping(HRegionInfo hri) {
-    HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
-    Set<HRegionInfo> replicas = defaultReplicaToOtherReplicas.get(defaultReplica);
-    if (replicas != null) {
-      replicas.remove(hri);
-      if (replicas.isEmpty()) {
-        defaultReplicaToOtherReplicas.remove(defaultReplica);
-      }
-    }
-  }
-
-  /**
-   * A dead server's wals have been split so that all the regions
-   * used to be open on it can be safely assigned now. Mark them assignable.
-   */
-  public synchronized void logSplit(final ServerName serverName) {
-    for (Iterator<Map.Entry<String, ServerName>> it
-        = lastAssignments.entrySet().iterator(); it.hasNext();) {
-      Map.Entry<String, ServerName> e = it.next();
-      if (e.getValue().equals(serverName)) {
-        it.remove();
-      }
-    }
-    long now = System.currentTimeMillis();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Adding to log splitting servers " + serverName);
-    }
-    processedServers.put(serverName, Long.valueOf(now));
-    Configuration conf = server.getConfiguration();
-    long obsoleteTime = conf.getLong(LOG_SPLIT_TIME, DEFAULT_LOG_SPLIT_TIME);
-    // Doesn't have to be very accurate about the clean up time
-    if (now > lastProcessedServerCleanTime + obsoleteTime) {
-      lastProcessedServerCleanTime = now;
-      long cutoff = now - obsoleteTime;
-      for (Iterator<Map.Entry<ServerName, Long>> it
-          = processedServers.entrySet().iterator(); it.hasNext();) {
-        Map.Entry<ServerName, Long> e = it.next();
-        if (e.getValue().longValue() < cutoff) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Removed from log splitting servers " + e.getKey());
-          }
-          it.remove();
-        }
-      }
-    }
-  }
-
-  /**
-   * Log split is done for a given region, so it is assignable now.
-   */
-  public void logSplit(final HRegionInfo region) {
-    clearLastAssignment(region);
-  }
-
-  public synchronized void clearLastAssignment(final HRegionInfo region) {
-    lastAssignments.remove(region.getEncodedName());
-  }
-
-  /**
-   * A region is offline, won't be in transition any more.
-   */
-  public void regionOffline(final HRegionInfo hri) {
-    regionOffline(hri, null);
-  }
-
-  /**
-   * A region is offline, won't be in transition any more. Its state
-   * should be the specified expected state, which can only be
-   * Split/Merged/Offline/null(=Offline)/SplittingNew/MergingNew.
-   */
-  public void regionOffline(
-      final HRegionInfo hri, final State expectedState) {
-    Preconditions.checkArgument(expectedState == null
-      || RegionState.isUnassignable(expectedState),
-        "Offlined region should not be " + expectedState);
-    if (isRegionInState(hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
-      // Remove it from all region maps
-      deleteRegion(hri);
-      return;
-    }
-    State newState =
-      expectedState == null ? State.OFFLINE : expectedState;
-    updateRegionState(hri, newState);
-    String encodedName = hri.getEncodedName();
-    synchronized (this) {
-      regionsInTransition.remove(encodedName);
-      ServerName oldServerName = regionAssignments.remove(hri);
-      if (oldServerName != null && serverHoldings.containsKey(oldServerName)) {
-        if (newState == State.MERGED || newState == State.SPLIT
-            || hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(),
-              TableState.State.DISABLED, TableState.State.DISABLING)) {
-          // Offline the region only if it's merged/split, or the table is disabled/disabling.
-          // Otherwise, offline it from this server only when it is online on a different server.
-          LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
-          removeFromServerHoldings(oldServerName, hri);
-          removeFromReplicaMapping(hri);
-        } else {
-          // Need to remember it so that we can offline it from this
-          // server when it is online on a different server.
-          oldAssignments.put(encodedName, oldServerName);
-        }
-      }
-    }
-  }
-
-  /**
-   * A server is offline, all regions on it are dead.
-   */
-  public List<HRegionInfo> serverOffline(final ServerName sn) {
-    // Offline all regions on this server not already in transition.
-    List<HRegionInfo> rits = new ArrayList<>();
-    Set<HRegionInfo> regionsToCleanIfNoMetaEntry = new HashSet<>();
-    // Offline regions outside the loop and synchronized block to avoid
-    // ConcurrentModificationException and deadlock in case of meta anassigned,
-    // but RegionState a blocked.
-    Set<HRegionInfo> regionsToOffline = new HashSet<>();
-    synchronized (this) {
-      Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
-      if (assignedRegions == null) {
-        assignedRegions = new HashSet<>();
-      }
-
-      for (HRegionInfo region : assignedRegions) {
-        // Offline open regions, no need to offline if SPLIT/MERGED/OFFLINE
-        if (isRegionOnline(region)) {
-          regionsToOffline.add(region);
-        } else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
-          LOG.debug("Offline splitting/merging region " + getRegionState(region));
-          regionsToOffline.add(region);
-        }
-      }
-
-      for (RegionState state : regionsInTransition.values()) {
-        HRegionInfo hri = state.getRegion();
-        if (assignedRegions.contains(hri)) {
-          // Region is open on this region server, but in transition.
-          // This region must be moving away from this server, or splitting/merging.
-          // SSH will handle it, either skip assigning, or re-assign.
-          LOG.info("Transitioning " + state + " will be handled by ServerCrashProcedure for " + sn);
-        } else if (sn.equals(state.getServerName())) {
-          // Region is in transition on this region server, and this
-          // region is not open on this server. So the region must be
-          // moving to this server from another one (i.e. opening or
-          // pending open on this server, was open on another one.
-          // Offline state is also kind of pending open if the region is in
-          // transition. The region could be in failed_close state too if we have
-          // tried several times to open it while this region server is not reachable)
-          if (isOneOfStates(state, State.OPENING, State.PENDING_OPEN,
-              State.FAILED_OPEN, State.FAILED_CLOSE, State.OFFLINE)) {
-            LOG.info("Found region in " + state +
-              " to be reassigned by ServerCrashProcedure for " + sn);
-            rits.add(hri);
-          } else if (isOneOfStates(state, State.SPLITTING_NEW, State.MERGING_NEW)) {
-            regionsToCleanIfNoMetaEntry.add(state.getRegion());
-          } else {
-            LOG.warn("THIS SHOULD NOT HAPPEN: unexpected " + state);
-          }
-        }
-      }
-      this.notifyAll();
-    }
-
-    for (HRegionInfo hri : regionsToOffline) {
-      regionOffline(hri);
-    }
-
-    cleanIfNoMetaEntry(regionsToCleanIfNoMetaEntry);
-    return rits;
-  }
-
-  /**
-   * This method does an RPC to hbase:meta. Do not call this method with a lock/synchronize held.
-   * @param hris The hris to check if empty in hbase:meta and if so, clean them up.
-   */
-  private void cleanIfNoMetaEntry(Set<HRegionInfo> hris) {
-    if (hris.isEmpty()) return;
-    for (HRegionInfo hri: hris) {
-      try {
-        // This is RPC to meta table. It is done while we have a synchronize on
-        // regionstates. No progress will be made if meta is not available at this time.
-        // This is a cleanup task. Not critical.
-        if (MetaTableAccessor.getRegion(server.getConnection(), hri.getEncodedNameAsBytes()) ==
-            null) {
-          regionOffline(hri);
-          FSUtils.deleteRegionDir(server.getConfiguration(), hri);
-        }
-      } catch (IOException e) {
-        LOG.warn("Got exception while deleting " + hri + " directories from file system.", e);
-      }
-    }
-  }
-
-  /**
-   * Gets the online regions of the specified table.
-   * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
-   * Only returns <em>online</em> regions.  If a region on this table has been
-   * closed during a disable, etc., it will be included in the returned list.
-   * So, the returned list may not necessarily be ALL regions in this table, its
-   * all the ONLINE regions in the table.
-   * @param tableName
-   * @return Online regions from <code>tableName</code>
-   */
-  public synchronized List<HRegionInfo> getRegionsOfTable(TableName tableName) {
-    List<HRegionInfo> tableRegions = new ArrayList<>();
-    // boundary needs to have table's name but regionID 0 so that it is sorted
-    // before all table's regions.
-    HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
-    for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
-      if(!hri.getTable().equals(tableName)) break;
-      tableRegions.add(hri);
-    }
-    return tableRegions;
-  }
-
-  /**
-   * Gets current state of all regions of the table.
-   * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
-   * Method guaranteed to return keys for all states
-   * in {@link org.apache.hadoop.hbase.master.RegionState.State}
-   *
-   * @param tableName
-   * @return Online regions from <code>tableName</code>
-   */
-  public synchronized Map<RegionState.State, List<HRegionInfo>>
-  getRegionByStateOfTable(TableName tableName) {
-    Map<RegionState.State, List<HRegionInfo>> tableRegions = new HashMap<>();
-    for (State state : State.values()) {
-      tableRegions.put(state, new ArrayList<>());
-    }
-    Map<String, RegionState> indexMap = regionStatesTableIndex.get(tableName);
-    if (indexMap == null)
-      return tableRegions;
-    for (RegionState regionState : indexMap.values()) {
-      tableRegions.get(regionState.getState()).add(regionState.getRegion());
-    }
-    return tableRegions;
-  }
-
-  /**
-   * Wait on region to clear regions-in-transition.
-   * <p>
-   * If the region isn't in transition, returns immediately.  Otherwise, method
-   * blocks until the region is out of transition.
-   */
-  public synchronized void waitOnRegionToClearRegionsInTransition(
-      final HRegionInfo hri) throws InterruptedException {
-    if (!isRegionInTransition(hri)) return;
-
-    while(!server.isStopped() && isRegionInTransition(hri)) {
-      RegionState rs = getRegionState(hri);
-      LOG.info("Waiting on " + rs + " to clear regions-in-transition");
-      waitForUpdate(100);
-    }
-
-    if (server.isStopped()) {
-      LOG.info("Giving up wait on region in " +
-        "transition because stoppable.isStopped is set");
-    }
-  }
-
-  /**
-   * A table is deleted. Remove its regions from all internal maps.
-   * We loop through all regions assuming we don't delete tables too much.
-   */
-  public void tableDeleted(final TableName tableName) {
-    Set<HRegionInfo> regionsToDelete = new HashSet<>();
-    synchronized (this) {
-      for (RegionState state: regionStates.values()) {
-        HRegionInfo region = state.getRegion();
-        if (region.getTable().equals(tableName)) {
-          regionsToDelete.add(region);
-        }
-      }
-    }
-    for (HRegionInfo region: regionsToDelete) {
-      deleteRegion(region);
-    }
-  }
-
-  /**
-   * Get a copy of all regions assigned to a server
-   */
-  public synchronized Set<HRegionInfo> getServerRegions(ServerName serverName) {
-    Set<HRegionInfo> regions = serverHoldings.get(serverName);
-    if (regions == null) return null;
-    return new HashSet<>(regions);
-  }
-
-  /**
-   * Remove a region from all state maps.
-   */
-  @VisibleForTesting
-  public synchronized void deleteRegion(final HRegionInfo hri) {
-    String encodedName = hri.getEncodedName();
-    regionsInTransition.remove(encodedName);
-    regionStates.remove(encodedName);
-    TableName table = hri.getTable();
-    Map<String, RegionState> indexMap = regionStatesTableIndex.get(table);
-    indexMap.remove(encodedName);
-    if (indexMap.isEmpty())
-      regionStatesTableIndex.remove(table);
-    lastAssignments.remove(encodedName);
-    ServerName sn = regionAssignments.remove(hri);
-    if (sn != null) {
-      Set<HRegionInfo> regions = serverHoldings.get(sn);
-      regions.remove(hri);
-    }
-  }
-
-  /**
-   * Checking if a region was assigned to a server which is not online now.
-   * If so, we should hold re-assign this region till SSH has split its wals.
-   * Once logs are split, the last assignment of this region will be reset,
-   * which means a null last assignment server is ok for re-assigning.
-   *
-   * A region server could be dead but we don't know it yet. We may
-   * think it's online falsely. Therefore if a server is online, we still
-   * need to confirm it reachable and having the expected start code.
-   */
-  synchronized boolean wasRegionOnDeadServer(final String encodedName) {
-    ServerName server = lastAssignments.get(encodedName);
-    return isServerDeadAndNotProcessed(server);
-  }
-
-  synchronized boolean isServerDeadAndNotProcessed(ServerName server) {
-    if (server == null) return false;
-    if (serverManager.isServerOnline(server)) {
-      String hostAndPort = server.getHostAndPort();
-      long startCode = server.getStartcode();
-      Long deadCode = deadServers.get(hostAndPort);
-      if (deadCode == null || startCode > deadCode.longValue()) {
-        if (serverManager.isServerReachable(server)) {
-          return false;
-        }
-        // The size of deadServers won't grow unbounded.
-        deadServers.put(hostAndPort, Long.valueOf(startCode));
-      }
-      // Watch out! If the server is not dead, the region could
-      // remain unassigned. That's why ServerManager#isServerReachable
-      // should use some retry.
-      //
-      // We cache this info since it is very unlikely for that
-      // instance to come back up later on. We don't want to expire
-      // the server since we prefer to let it die naturally.
-      LOG.warn("Couldn't reach online server " + server);
-    }
-    // Now, we know it's dead. Check if it's processed
-    return !processedServers.containsKey(server);
-  }
-
- /**
-   * Get the last region server a region was on for purpose of re-assignment,
-   * i.e. should the re-assignment be held back till log split is done?
-   */
-  synchronized ServerName getLastRegionServerOfRegion(final String encodedName) {
-    return lastAssignments.get(encodedName);
-  }
-
-  synchronized void setLastRegionServerOfRegions(
-      final ServerName serverName, final List<HRegionInfo> regionInfos) {
-    for (HRegionInfo hri: regionInfos) {
-      setLastRegionServerOfRegion(serverName, hri.getEncodedName());
-    }
-  }
-
-  synchronized void setLastRegionServerOfRegion(
-      final ServerName serverName, final String encodedName) {
-    lastAssignments.put(encodedName, serverName);
-  }
-
-  synchronized boolean isRegionOnServer(
-      final HRegionInfo hri, final ServerName serverName) {
-    Set<HRegionInfo> regions = serverHoldings.get(serverName);
-    return regions == null ? false : regions.contains(hri);
-  }
-
-  public void prepareAssignDaughters(HRegionInfo a, HRegionInfo b) {
-     synchronized (this) {
-       if (isRegionInState(a, State.SPLITTING_NEW)) {
-         updateRegionState(a, State.OFFLINE, null);
-       }
-       if (isRegionInState(b, State.SPLITTING_NEW)) {
-         updateRegionState(b, State.OFFLINE, null);
-       }
-     }
-   }
-
-  public void prepareAssignMergedRegion(HRegionInfo mergedRegion) {
-    synchronized (this) {
-      if (isRegionInState(mergedRegion, State.MERGING_NEW)) {
-        updateRegionState(mergedRegion, State.OFFLINE, null);
-      }
-    }
-  }
-
-  void splitRegion(HRegionInfo p,
-      HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
-
-    regionStateStore.splitRegion(p, a, b, sn, getRegionReplication(p));
-    synchronized (this) {
-      // After PONR, split is considered to be done.
-      // Update server holdings to be aligned with the meta.
-      Set<HRegionInfo> regions = serverHoldings.get(sn);
-      if (regions == null) {
-        throw new IllegalStateException(sn + " should host some regions");
-      }
-      regions.remove(p);
-      regions.add(a);
-      regions.add(b);
-    }
-  }
-
-  void mergeRegions(HRegionInfo p,
-      HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
-    regionStateStore.mergeRegions(p, a, b, sn, getRegionReplication(a));
-    synchronized (this) {
-      // After PONR, merge is considered to be done.
-      // Update server holdings to be aligned with the meta.
-      Set<HRegionInfo> regions = serverHoldings.get(sn);
-      if (regions == null) {
-        throw new IllegalStateException(sn + " should host some regions");
-      }
-      regions.remove(a);
-      regions.remove(b);
-      regions.add(p);
-    }
-  }
-
-  private int getRegionReplication(HRegionInfo r) throws IOException {
-    if (tableStateManager != null) {
-      HTableDescriptor htd = server.getTableDescriptors().get(r.getTable());
-      if (htd != null) {
-        return htd.getRegionReplication();
-      }
-    }
-    return 1;
-  }
-
-  /**
-   * At cluster clean re/start, mark all user regions closed except those of tables
-   * that are excluded, such as disabled/disabling/enabling tables. All user regions
-   * and their previous locations are returned.
-   */
-  synchronized Map<HRegionInfo, ServerName> closeAllUserRegions(Set<TableName> excludedTables) {
-    boolean noExcludeTables = excludedTables == null || excludedTables.isEmpty();
-    Set<HRegionInfo> toBeClosed = new HashSet<>(regionStates.size());
-    for(RegionState state: regionStates.values()) {
-      HRegionInfo hri = state.getRegion();
-      if (state.isSplit() || hri.isSplit()) {
-        continue;
-      }
-      TableName tableName = hri.getTable();
-      if (!TableName.META_TABLE_NAME.equals(tableName)
-          && (noExcludeTables || !excludedTables.contains(tableName))) {
-        toBeClosed.add(hri);
-      }
-    }
-    Map<HRegionInfo, ServerName> allUserRegions = new HashMap<>(toBeClosed.size());
-    for (HRegionInfo hri: toBeClosed) {
-      RegionState regionState = updateRegionState(hri, State.CLOSED);
-      allUserRegions.put(hri, regionState.getServerName());
-    }
-    return allUserRegions;
-  }
-
-  /**
-   * Compute the average load across all region servers.
-   * Currently, this uses a very naive computation - just uses the number of
-   * regions being served, ignoring stats about number of requests.
-   * @return the average load
-   */
-  protected synchronized double getAverageLoad() {
-    int numServers = 0, totalLoad = 0;
-    for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
-      Set<HRegionInfo> regions = e.getValue();
-      ServerName serverName = e.getKey();
-      int regionCount = regions.size();
-      if (serverManager.isServerOnline(serverName)) {
-        totalLoad += regionCount;
-        numServers++;
-      }
-    }
-    if (numServers > 1) {
-      // The master region server holds only a couple regions.
-      // Don't consider this server in calculating the average load
-      // if there are other region servers to avoid possible confusion.
-      Set<HRegionInfo> hris = serverHoldings.get(server.getServerName());
-      if (hris != null) {
-        totalLoad -= hris.size();
-        numServers--;
-      }
-    }
-    return numServers == 0 ? 0.0 :
-      (double)totalLoad / (double)numServers;
-  }
-
-  protected Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable() {
-    return getAssignmentsByTable(false);
-  }
-
-  /**
-   * This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
-   * Can't let out original since it can change and at least the load balancer
-   * wants to iterate this exported list.  We need to synchronize on regions
-   * since all access to this.servers is under a lock on this.regions.
-   * @param forceByCluster a flag to force to aggregate the server-load to the cluster level
-   * @return A clone of current assignments by table.
-   */
-  protected Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable(
-          boolean forceByCluster) {
-    Map<TableName, Map<ServerName, List<HRegionInfo>>> result;
-    synchronized (this) {
-      result = getTableRSRegionMap(server.getConfiguration().getBoolean(
-              HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE,false) && !forceByCluster);
-    }
-    Map<ServerName, ServerLoad>
-            onlineSvrs = serverManager.getOnlineServers();
-    // Take care of servers w/o assignments, and remove servers in draining mode
-    List<ServerName> drainingServers = this.serverManager.getDrainingServersList();
-    for (Map<ServerName, List<HRegionInfo>> map: result.values()) {
-      for (ServerName svr: onlineSvrs.keySet()) {
-        if (!map.containsKey(svr)) {
-          map.put(svr, new ArrayList<>());
-        }
-      }
-      map.keySet().removeAll(drainingServers);
-    }
-    return result;
-  }
-
-  private Map<TableName, Map<ServerName, List<HRegionInfo>>> getTableRSRegionMap(Boolean bytable){
-    Map<TableName, Map<ServerName, List<HRegionInfo>>> result = new HashMap<>();
-    for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
-      for (HRegionInfo hri: e.getValue()) {
-        if (hri.isMetaRegion()) continue;
-        TableName tablename = bytable ? hri.getTable() : TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME);
-        Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
-        if (svrToRegions == null) {
-          svrToRegions = new HashMap<>(serverHoldings.size());
-          result.put(tablename, svrToRegions);
-        }
-        List<HRegionInfo> regions = svrToRegions.get(e.getKey());
-        if (regions == null) {
-          regions = new ArrayList<>();
-          svrToRegions.put(e.getKey(), regions);
-        }
-        regions.add(hri);
-      }
-    }
-    return result;
-  }
-
-  public RegionState getRegionState(final HRegionInfo hri) {
-    return getRegionState(hri.getEncodedName());
-  }
-
-  /**
-   * Returns a clone of region assignments per server
-   * @return a Map of ServerName to a List of HRegionInfo's
-   */
-  protected synchronized Map<ServerName, List<HRegionInfo>> getRegionAssignmentsByServer() {
-    Map<ServerName, List<HRegionInfo>> regionsByServer = new HashMap<>(serverHoldings.size());
-    for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
-      regionsByServer.put(e.getKey(), new ArrayList<>(e.getValue()));
-    }
-    return regionsByServer;
-  }
-
-  public synchronized RegionState getRegionState(final String encodedName) {
-    return regionStates.get(encodedName);
-  }
-
-  /**
-   * Get the HRegionInfo from cache, if not there, from the hbase:meta table.
-   * Be careful. Does RPC. Do not hold a lock or synchronize when you call this method.
-   * @param  regionName
-   * @return HRegionInfo for the region
-   */
-  @SuppressWarnings("deprecation")
-  protected HRegionInfo getRegionInfo(final byte [] regionName) {
-    String encodedName = HRegionInfo.encodeRegionName(regionName);
-    RegionState regionState = getRegionState(encodedName);
-    if (regionState != null) {
-      return regionState.getRegion();
-    }
-
-    try {
-      Pair<HRegionInfo, ServerName> p =
-        MetaTableAccessor.getRegion(server.getConnection(), regionName);
-      HRegionInfo hri = p == null ? null : p.getFirst();
-      if (hri != null) {
-        createRegionState(hri);
-      }
-      return hri;
-    } catch (IOException e) {
-      server.abort("Aborting because error occurred while reading "
-        + Bytes.toStringBinary(regionName) + " from hbase:meta", e);
-      return null;
-    }
-  }
-
-  static boolean isOneOfStates(RegionState regionState, State... states) {
-    State s = regionState != null ? regionState.getState() : null;
-    for (State state: states) {
-      if (s == state) return true;
-    }
-    return false;
-  }
-
-  /**
-   * Update a region state. It will be put in transition if not already there.
-   */
-  private RegionState updateRegionState(final HRegionInfo hri,
-      final RegionState.State state, final ServerName serverName, long openSeqNum) {
-    if (state == RegionState.State.FAILED_CLOSE || state == RegionState.State.FAILED_OPEN) {
-      LOG.warn("Failed to open/close " + hri.getShortNameToLog()
-        + " on " + serverName + ", set to " + state);
-    }
-
-    String encodedName = hri.getEncodedName();
-    RegionState regionState = new RegionState(
-      hri, state, System.currentTimeMillis(), serverName);
-    RegionState oldState = getRegionState(encodedName);
-    if (!regionState.equals(oldState)) {
-      LOG.info("Transition " + oldState + " to " + regionState);
-      // Persist region state before updating in-memory info, if needed
-      regionStateStore.updateRegionState(openSeqNum, regionState, oldState);
-    }
-
-    synchronized (this) {
-      RegionState oldRegionState = regionsInTransition.put(encodedName, regionState);
-      // When region transform old region state to new region state,
-      // accumulate the RIT duration to new region state.
-      if (oldRegionState != null) {
-        regionState.updateRitDuration(oldRegionState.getStamp());
-      }
-      putRegionState(regionState);
-
-      // For these states, region should be properly closed.
-      // There should be no log splitting issue.
-      if ((state == State.CLOSED || state == State.MERGED
-          || state == State.SPLIT) && lastAssignments.containsKey(encodedName)) {
-        ServerName last = lastAssignments.get(encodedName);
-        if (last.equals(serverName)) {
-          lastAssignments.remove(encodedName);
-        } else {
-          LOG.warn(encodedName + " moved to " + state + " on "
-            + serverName + ", expected " + last);
-        }
-      }
-
-      // Once a region is opened, record its last assignment right away.
-      if (serverName != null && state == State.OPEN) {
-        ServerName last = lastAssignments.get(encodedName);
-        if (!serverName.equals(last)) {
-          lastAssignments.put(encodedName, serverName);
-          if (last != null && isServerDeadAndNotProcessed(last)) {
-            LOG.warn(encodedName + " moved to " + serverName
-              + ", while it's previous host " + last
-              + " is dead but not processed yet");
-          }
-        }
-      }
-
-      // notify the change
-      this.notifyAll();
-    }
-    return regionState;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index db0a0e5..dfebc7f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -57,12 +57,10 @@ import org.apache.hadoop.hbase.ipc.FailedServerException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -314,7 +312,8 @@ public class ServerManager {
     }
   }
 
-  void regionServerReport(ServerName sn,
+  @VisibleForTesting
+  public void regionServerReport(ServerName sn,
       ServerLoad sl) throws YouAreDeadException {
     checkIsDead(sn, "REPORT");
     if (null == this.onlineServers.replace(sn, sl)) {
@@ -614,12 +613,7 @@ public class ServerManager {
       return;
     }
 
-    boolean carryingMeta = master.getAssignmentManager().isCarryingMeta(serverName);
-    ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
-    procExec.submitProcedure(new ServerCrashProcedure(
-      procExec.getEnvironment(), serverName, true, carryingMeta));
-    LOG.debug("Added=" + serverName +
-      " to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
+    master.getAssignmentManager().submitServerCrash(serverName, true);
 
     // Tell our listeners that a server was removed
     if (!this.listeners.isEmpty()) {
@@ -629,6 +623,37 @@ public class ServerManager {
     }
   }
 
+  /**
+   * Sends an MERGE REGIONS RPC to the specified server to merge the specified
+   * regions.
+   * <p>
+   * A region server could reject the close request because it either does not
+   * have the specified region.
+   * @param server server to merge regions
+   * @param region_a region to merge
+   * @param region_b region to merge
+   * @param forcible true if do a compulsory merge, otherwise we will only merge
+   *          two adjacent regions
+   * @throws IOException
+   */
+  public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
+      HRegionInfo region_b, boolean forcible, final User user) throws IOException {
+    if (server == null)
+      throw new NullPointerException("Passed server is null");
+    if (region_a == null || region_b == null)
+      throw new NullPointerException("Passed region is null");
+    AdminService.BlockingInterface admin = getRsAdmin(server);
+    if (admin == null) {
+      throw new IOException("Attempting to send MERGE REGIONS RPC to server "
+          + server.toString() + " for region "
+          + region_a.getRegionNameAsString() + ","
+          + region_b.getRegionNameAsString()
+          + " failed because no RPC connection found to this server");
+    }
+    HBaseRpcController controller = newRpcController();
+    ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible, user);
+  }
+
   @VisibleForTesting
   public void moveFromOnlineToDeadServers(final ServerName sn) {
     synchronized (onlineServers) {
@@ -660,9 +685,7 @@ public class ServerManager {
     }
 
     this.deadservers.add(serverName);
-    ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
-    procExec.submitProcedure(new ServerCrashProcedure(
-      procExec.getEnvironment(), serverName, shouldSplitWal, false));
+    master.getAssignmentManager().submitServerCrash(serverName, shouldSplitWal);
   }
 
   /**
@@ -877,30 +900,6 @@ public class ServerManager {
   }
 
   /**
-   * Sends an CLOSE RPC to the specified server to close the specified region for SPLIT.
-   * <p>
-   * A region server could reject the close request because it either does not
-   * have the specified region or the region is being split.
-   * @param server server to close a region
-   * @param regionToClose the info of the region(s) to close
-   * @throws IOException
-   */
-  public boolean sendRegionCloseForSplitOrMerge(
-      final ServerName server,
-      final HRegionInfo... regionToClose) throws IOException {
-    if (server == null) {
-      throw new NullPointerException("Passed server is null");
-    }
-    AdminService.BlockingInterface admin = getRsAdmin(server);
-    if (admin == null) {
-      throw new IOException("Attempting to send CLOSE For Split or Merge RPC to server " +
-        server.toString() + " failed because no RPC connection found to this server.");
-    }
-    HBaseRpcController controller = newRpcController();
-    return ProtobufUtil.closeRegionForSplitOrMerge(controller, admin, server, regionToClose);
-  }
-
-  /**
    * Sends a WARMUP RPC to the specified server to warmup the specified region.
    * <p>
    * A region server could reject the close request because it either does not
@@ -990,7 +989,7 @@ public class ServerManager {
     * @throws IOException
     * @throws RetriesExhaustedException wrapping a ConnectException if failed
     */
-  private AdminService.BlockingInterface getRsAdmin(final ServerName sn)
+  public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
   throws IOException {
     AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
     if (admin == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index 7582d42..4a2c942 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -313,8 +313,9 @@ public class TableNamespaceManager {
   }
 
   private boolean isTableAssigned() {
-    return !masterServices.getAssignmentManager()
-        .getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
+    // TODO: we have a better way now (wait on event)
+    return masterServices.getAssignmentManager()
+        .getRegionStates().hasTableRegionStates(TableName.NAMESPACE_TABLE_NAME);
   }
 
   public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java
deleted file mode 100644
index ccff6f0..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java
+++ /dev/null
@@ -1,47 +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;
-
-import java.util.concurrent.Callable;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-
-/**
- * A callable object that invokes the corresponding action that needs to be
- * taken for unassignment of a region in transition. Implementing as future
- * callable we are able to act on the timeout asynchronously.
- */
-@InterfaceAudience.Private
-public class UnAssignCallable implements Callable<Object> {
-  private AssignmentManager assignmentManager;
-
-  private HRegionInfo hri;
-
-  public UnAssignCallable(AssignmentManager assignmentManager, HRegionInfo hri) {
-    this.assignmentManager = assignmentManager;
-    this.hri = hri;
-  }
-
-  @Override
-  public Object call() throws Exception {
-    assignmentManager.unassign(hri);
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
new file mode 100644
index 0000000..4105ffd
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
@@ -0,0 +1,270 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AssignRegionStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+/**
+ * Procedure that describe the assignment of a single region.
+ * There can only be one RegionTransitionProcedure per region running at a time
+ * since each procedure takes a lock on the region.
+ *
+ * <p>The Assign starts by pushing the "assign" operation to the AssignmentManager
+ * and then will go in a "waiting" state.
+ * The AM will batch the "assign" requests and ask the Balancer where to put
+ * the region (the various policies will be respected: retain, round-robin, random).
+ * Once the AM and the balancer have found a place for the region the procedure
+ * will be resumed and an "open region" request will be placed in the Remote Dispatcher
+ * queue, and the procedure once again will go in a "waiting state".
+ * The Remote Dispatcher will batch the various requests for that server and
+ * they will be sent to the RS for execution.
+ * The RS will complete the open operation by calling master.reportRegionStateTransition().
+ * The AM will intercept the transition report, and notify the procedure.
+ * The procedure will finish the assignment by publishing to new state on meta
+ * or it will retry the assignment.
+ * <p>This procedure does not rollback when beyond the first
+ * REGION_TRANSITION_QUEUE step; it will press on trying to assign in the face of
+ * failure.
+ */
+@InterfaceAudience.Private
+public class AssignProcedure extends RegionTransitionProcedure {
+  private static final Log LOG = LogFactory.getLog(AssignProcedure.class);
+
+  private ServerName targetServer = null;
+  private boolean forceNewPlan = false;
+
+  public AssignProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  public AssignProcedure(final HRegionInfo regionInfo) {
+    this(regionInfo, false);
+  }
+
+  public AssignProcedure(final HRegionInfo regionInfo, final boolean forceNewPlan) {
+    super(regionInfo);
+    this.forceNewPlan = forceNewPlan;
+    this.targetServer = null;
+  }
+
+  public AssignProcedure(final HRegionInfo regionInfo, final ServerName targetServer) {
+    super(regionInfo);
+    this.forceNewPlan = false;
+    this.targetServer = targetServer;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.ASSIGN;
+  }
+
+  @Override
+  protected boolean isRollbackSupported(final RegionTransitionState state) {
+    switch (state) {
+      case REGION_TRANSITION_QUEUE:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    final AssignRegionStateData.Builder state = AssignRegionStateData.newBuilder()
+        .setTransitionState(getTransitionState())
+        .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
+    if (forceNewPlan) {
+      state.setForceNewPlan(true);
+    }
+    if (targetServer != null) {
+      state.setTargetServer(ProtobufUtil.toServerName(targetServer));
+    }
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    final AssignRegionStateData state = AssignRegionStateData.parseDelimitedFrom(stream);
+    setTransitionState(state.getTransitionState());
+    setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
+    forceNewPlan = state.getForceNewPlan();
+    if (state.hasTargetServer()) {
+      targetServer = ProtobufUtil.toServerName(state.getTargetServer());
+    }
+  }
+
+  @Override
+  protected boolean startTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
+      throws IOException {
+    LOG.info("Start " + regionNode);
+    // If the region is already open we can't do much...
+    if (regionNode.isInState(State.OPEN) && isServerOnline(env, regionNode)) {
+      LOG.info("Already assigned: " + regionNode);
+      return false;
+    }
+
+    // If we haven't started the operation yet, we can abort
+    if (aborted.get() && regionNode.isInState(State.CLOSED, State.OFFLINE)) {
+      if (incrementAndCheckMaxAttempts(env, regionNode)) {
+        regionNode.setState(State.FAILED_OPEN);
+        setFailure(getClass().getSimpleName(),
+          new RetriesExhaustedException("Max attempts exceeded"));
+      } else {
+        setAbortFailure(getClass().getSimpleName(), "Abort requested");
+      }
+      return false;
+    }
+
+    // send assign (add into assign-pool). region is now in OFFLINE state
+    regionNode.setState(State.OFFLINE);
+    if (forceNewPlan) {
+      regionNode.setRegionLocation(null);
+    } else if (targetServer != null) {
+      regionNode.setRegionLocation(targetServer);
+    }
+
+    env.getAssignmentManager().queueAssign(regionNode);
+    return true;
+  }
+
+  @Override
+  protected boolean updateTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
+      throws IOException, ProcedureSuspendedException {
+    // TODO: crash if targetServer is specified and not online
+    // which is also the case when the balancer provided us with a different location.
+    LOG.info("Update " + regionNode);
+    if (regionNode.getRegionLocation() == null) {
+      setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
+      return true;
+    }
+
+    if (!isServerOnline(env, regionNode)) {
+      // TODO: is this correct? should we wait the chore/ssh?
+      LOG.info("Server not online: " + regionNode);
+      setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
+      return true;
+    }
+
+    // Wait until server reported. If we have resumed the region may already be assigned.
+    if (env.getAssignmentManager().waitServerReportEvent(regionNode.getRegionLocation(), this)) {
+      throw new ProcedureSuspendedException();
+    }
+
+    if (regionNode.isInState(State.OPEN)) {
+      LOG.info("Already assigned: " + regionNode);
+      return false;
+    }
+
+    // region is now in OPENING state
+    env.getAssignmentManager().markRegionAsOpening(regionNode);
+
+    // TODO: Requires a migration to be open by the RS?
+    // regionNode.getFormatVersion()
+
+    // Add the open region operation to the server dispatch queue.
+    // The pending open will be dispatched to the server together with the other
+    // pending operation for that server.
+    addToRemoteDispatcher(env, regionNode.getRegionLocation());
+    return true;
+  }
+
+  @Override
+  protected void completeTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
+      throws IOException {
+    LOG.info("Completed " + regionNode);
+    env.getAssignmentManager().markRegionAsOpened(regionNode);
+  }
+
+  @Override
+  protected void reportTransition(final MasterProcedureEnv env, final RegionStateNode regionNode,
+      final TransitionCode code, final long openSeqNum) throws UnexpectedStateException {
+    switch (code) {
+      case OPENED:
+        LOG.debug("Report " + code + " openSeqNum=" + openSeqNum + ", " + regionNode);
+        if (openSeqNum < 0) {
+          throw new UnexpectedStateException("Report unexpected " + code + " transition openSeqNum=" +
+              openSeqNum + ", " + regionNode);
+        }
+        regionNode.setOpenSeqNum(openSeqNum);
+        setTransitionState(RegionTransitionState.REGION_TRANSITION_FINISH);
+        break;
+      case FAILED_OPEN:
+        LOG.warn("Report " + code + " openSeqNum=" + openSeqNum + ", " + regionNode);
+        handleFailure(env, regionNode);
+        break;
+      default:
+        throw new UnexpectedStateException("Report unexpected " + code +
+            " transition openSeqNum=" + openSeqNum + ", " + regionNode +
+            ", expected OPENED or FAILED_OPEN.");
+    }
+  }
+
+  private void handleFailure(final MasterProcedureEnv env, final RegionStateNode regionNode) {
+    if (incrementAndCheckMaxAttempts(env, regionNode)) {
+      aborted.set(true);
+    }
+    regionNode.setState(State.OFFLINE);
+    regionNode.setRegionLocation(null);
+    setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
+  }
+
+  private boolean incrementAndCheckMaxAttempts(final MasterProcedureEnv env,
+      final RegionStateNode regionNode) {
+    final int retries = env.getAssignmentManager().getRegionStates()
+        .addToFailedOpen(regionNode).incrementAndGetRetries();
+      LOG.info("RETRY " + retries + ": " + this);
+      return retries >= env.getAssignmentManager().getAssignMaxAttempts();
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(final MasterProcedureEnv env, final ServerName serverName) {
+    assert serverName.equals(getRegionState(env).getRegionLocation());
+    return new RegionOpenOperation(this, getRegionInfo(),
+        env.getAssignmentManager().getFavoredNodes(getRegionInfo()), false);
+  }
+
+  @Override
+  protected void remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
+      final IOException exception) {
+    // TODO: put the server in the bad list?
+    handleFailure(env, regionNode);
+  }
+}


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 484decc..68ed7e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -19,28 +19,23 @@ package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InterruptedIOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
-import java.util.concurrent.locks.Lock;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.MasterWalManager;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
@@ -48,10 +43,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.zookeeper.KeeperException;
 
 /**
  * Handle crashed server. This is a port to ProcedureV2 of what used to be euphemistically called
@@ -65,9 +57,6 @@ import org.apache.zookeeper.KeeperException;
  * completion of each successful flow step. We do this so that we do not 'deadlock' waiting on
  * a region assignment so we can replay edits which could happen if a region moved there are edits
  * on two servers for replay.
- *
- * <p>TODO: ASSIGN and WAIT_ON_ASSIGN (at least) are not idempotent. Revisit when assign is pv2.
- * TODO: We do not have special handling for system tables.
  */
 public class ServerCrashProcedure
 extends StateMachineProcedure<MasterProcedureEnv, ServerCrashState>
@@ -75,36 +64,6 @@ implements ServerProcedureInterface {
   private static final Log LOG = LogFactory.getLog(ServerCrashProcedure.class);
 
   /**
-   * Configuration key to set how long to wait in ms doing a quick check on meta state.
-   */
-  public static final String KEY_SHORT_WAIT_ON_META =
-      "hbase.master.servercrash.short.wait.on.meta.ms";
-
-  public static final int DEFAULT_SHORT_WAIT_ON_META = 1000;
-
-  /**
-   * Configuration key to set how many retries to cycle before we give up on meta.
-   * Each attempt will wait at least {@link #KEY_SHORT_WAIT_ON_META} milliseconds.
-   */
-  public static final String KEY_RETRIES_ON_META =
-      "hbase.master.servercrash.meta.retries";
-
-  public static final int DEFAULT_RETRIES_ON_META = 10;
-
-  /**
-   * Configuration key to set how long to wait in ms on regions in transition.
-   */
-  public static final String KEY_WAIT_ON_RIT =
-      "hbase.master.servercrash.wait.on.rit.ms";
-
-  public static final int DEFAULT_WAIT_ON_RIT = 30000;
-
-  private static final Set<HRegionInfo> META_REGION_SET = new HashSet<>();
-  static {
-    META_REGION_SET.add(HRegionInfo.FIRST_META_REGIONINFO);
-  }
-
-  /**
    * Name of the crashed server to process.
    */
   private ServerName serverName;
@@ -117,14 +76,8 @@ implements ServerProcedureInterface {
   /**
    * Regions that were on the crashed server.
    */
-  private Set<HRegionInfo> regionsOnCrashedServer;
-
-  /**
-   * Regions assigned. Usually some subset of {@link #regionsOnCrashedServer}.
-   */
-  private List<HRegionInfo> regionsAssigned;
+  private List<HRegionInfo> regionsOnCrashedServer;
 
-  private boolean distributedLogReplay = false;
   private boolean carryingMeta = false;
   private boolean shouldSplitWal;
 
@@ -175,7 +128,7 @@ implements ServerProcedureInterface {
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, ServerCrashState state)
-      throws ProcedureYieldException {
+      throws ProcedureSuspendedException, ProcedureYieldException {
     if (LOG.isTraceEnabled()) {
       LOG.trace(state);
     }
@@ -186,11 +139,7 @@ implements ServerProcedureInterface {
     } else {
       this.cycles++;
     }
-    MasterServices services = env.getMasterServices();
-    // Is master fully online? If not, yield. No processing of servers unless master is up
-    if (!services.getAssignmentManager().isFailoverCleanupDone()) {
-      throwProcedureYieldException("Waiting on master failover to complete");
-    }
+    final MasterServices services = env.getMasterServices();
     // HBASE-14802
     // If we have not yet notified that we are processing a dead server, we should do now.
     if (!notifiedDeadServer) {
@@ -204,95 +153,48 @@ implements ServerProcedureInterface {
         LOG.info("Start processing crashed " + this.serverName);
         start(env);
         // If carrying meta, process it first. Else, get list of regions on crashed server.
-        if (this.carryingMeta) setNextState(ServerCrashState.SERVER_CRASH_PROCESS_META);
-        else setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
+        if (this.carryingMeta) {
+          setNextState(ServerCrashState.SERVER_CRASH_PROCESS_META);
+        } else {
+          setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
+        }
         break;
 
       case SERVER_CRASH_GET_REGIONS:
         // If hbase:meta is not assigned, yield.
-        if (!isMetaAssignedQuickTest(env)) {
-          // isMetaAssignedQuickTest does not really wait. Let's delay a little before
-          // another round of execution.
-          long wait =
-              env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META,
-                DEFAULT_SHORT_WAIT_ON_META);
-          wait = wait / 10;
-          Thread.sleep(wait);
-          throwProcedureYieldException("Waiting on hbase:meta assignment");
+        if (env.getAssignmentManager().waitMetaInitialized(this)) {
+          throw new ProcedureSuspendedException();
         }
-        this.regionsOnCrashedServer =
-            services.getAssignmentManager().getRegionStates().getServerRegions(this.serverName);
-        // Where to go next? Depends on whether we should split logs at all or if we should do
-        // distributed log splitting (DLS) vs distributed log replay (DLR).
+
+        this.regionsOnCrashedServer = services.getAssignmentManager().getRegionStates()
+          .getServerRegionInfoSet(serverName);
+        LOG.info("GET REGIONS ON CRASHED SERVER: " + regionsOnCrashedServer);
+        // Where to go next? Depends on whether we should split logs at all or
+        // if we should do distributed log splitting.
         if (!this.shouldSplitWal) {
           setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
-        } else if (this.distributedLogReplay) {
-          setNextState(ServerCrashState.SERVER_CRASH_PREPARE_LOG_REPLAY);
         } else {
           setNextState(ServerCrashState.SERVER_CRASH_SPLIT_LOGS);
         }
         break;
 
       case SERVER_CRASH_PROCESS_META:
-        // If we fail processing hbase:meta, yield.
-        if (!processMeta(env)) {
-          throwProcedureYieldException("Waiting on regions-in-transition to clear");
-        }
+        processMeta(env);
         setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
         break;
 
-      case SERVER_CRASH_PREPARE_LOG_REPLAY:
-        prepareLogReplay(env, this.regionsOnCrashedServer);
-        setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
-        break;
-
       case SERVER_CRASH_SPLIT_LOGS:
         splitLogs(env);
-        // If DLR, go to FINISH. Otherwise, if DLS, go to SERVER_CRASH_CALC_REGIONS_TO_ASSIGN
-        if (this.distributedLogReplay) setNextState(ServerCrashState.SERVER_CRASH_FINISH);
-        else setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
+        setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
         break;
 
       case SERVER_CRASH_ASSIGN:
-        List<HRegionInfo> regionsToAssign = calcRegionsToAssign(env);
-
-        // Assign may not be idempotent. SSH used to requeue the SSH if we got an IOE assigning
-        // which is what we are mimicing here but it looks prone to double assignment if assign
-        // fails midway. TODO: Test.
-
         // If no regions to assign, skip assign and skip to the finish.
-        boolean regions = regionsToAssign != null && !regionsToAssign.isEmpty();
-        if (regions) {
-          this.regionsAssigned = regionsToAssign;
-          if (!assign(env, regionsToAssign)) {
-            throwProcedureYieldException("Failed assign; will retry");
-          }
-        }
-        if (this.shouldSplitWal && distributedLogReplay) {
-          // Take this route even if there are apparently no regions assigned. This may be our
-          // second time through here; i.e. we assigned and crashed just about here. On second
-          // time through, there will be no regions because we assigned them in the previous step.
-          // Even though no regions, we need to go through here to clean up the DLR zk markers.
-          setNextState(ServerCrashState.SERVER_CRASH_WAIT_ON_ASSIGN);
-        } else {
-          setNextState(ServerCrashState.SERVER_CRASH_FINISH);
-        }
-        break;
-
-      case SERVER_CRASH_WAIT_ON_ASSIGN:
-        // TODO: The list of regionsAssigned may be more than we actually assigned. See down in
-        // AM #1629 around 'if (regionStates.wasRegionOnDeadServer(encodedName)) {' where where we
-        // will skip assigning a region because it is/was on a dead server. Should never happen!
-        // It was on this server. Worst comes to worst, we'll still wait here till other server is
-        // processed.
-
-        // If the wait on assign failed, yield -- if we have regions to assign.
-        if (this.regionsAssigned != null && !this.regionsAssigned.isEmpty()) {
-          if (!waitOnAssign(env, this.regionsAssigned)) {
-            throwProcedureYieldException("Waiting on region assign");
-          }
+        if (filterDefaultMetaRegions(regionsOnCrashedServer)) {
+          addChildProcedure(env.getAssignmentManager()
+            .createAssignProcedures(regionsOnCrashedServer, true));
         }
-        setNextState(ServerCrashState.SERVER_CRASH_SPLIT_LOGS);
+        setNextState(ServerCrashState.SERVER_CRASH_FINISH);
         break;
 
       case SERVER_CRASH_FINISH:
@@ -305,10 +207,6 @@ implements ServerProcedureInterface {
       }
     } catch (IOException e) {
       LOG.warn("Failed serverName=" + this.serverName + ", state=" + state + "; retry", e);
-    } catch (InterruptedException e) {
-      // TODO: Make executor allow IEs coming up out of execute.
-      LOG.warn("Interrupted serverName=" + this.serverName + ", state=" + state + "; retry", e);
-      Thread.currentThread().interrupt();
     }
     return Flow.HAS_MORE_STATE;
   }
@@ -319,84 +217,54 @@ implements ServerProcedureInterface {
    * @throws IOException
    */
   private void start(final MasterProcedureEnv env) throws IOException {
-    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+    final MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
     // Set recovery mode late. This is what the old ServerShutdownHandler used do.
     mwm.setLogRecoveryMode();
-    this.distributedLogReplay = mwm.getLogRecoveryMode() == RecoveryMode.LOG_REPLAY;
+    if (mwm.getLogRecoveryMode() == RecoveryMode.LOG_REPLAY) {
+      String msg = "distributed log replay is not supported";
+      LOG.error(msg);
+      env.getMasterServices().abort(msg, null);
+    }
   }
 
   /**
    * @param env
-   * @return False if we fail to assign and split logs on meta ('process').
    * @throws IOException
    * @throws InterruptedException
    */
-  private boolean processMeta(final MasterProcedureEnv env)
-  throws IOException {
+  private void processMeta(final MasterProcedureEnv env) throws IOException {
     if (LOG.isDebugEnabled()) LOG.debug("Processing hbase:meta that was on " + this.serverName);
-    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    HRegionInfo metaHRI = HRegionInfo.FIRST_META_REGIONINFO;
+
     if (this.shouldSplitWal) {
-      if (this.distributedLogReplay) {
-        prepareLogReplay(env, META_REGION_SET);
-      } else {
-        // TODO: Matteo. We BLOCK here but most important thing to be doing at this moment.
-        mwm.splitMetaLog(serverName);
-        am.getRegionStates().logSplit(metaHRI);
-      }
+      // TODO: Matteo. We BLOCK here but most important thing to be doing at this moment.
+      env.getMasterServices().getMasterWalManager().splitMetaLog(serverName);
     }
 
     // Assign meta if still carrying it. Check again: region may be assigned because of RIT timeout
-    boolean processed = true;
-    if (am.isCarryingMeta(serverName)) {
-      // TODO: May block here if hard time figuring state of meta.
-      am.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
-      verifyAndAssignMetaWithRetries(env);
-      if (this.shouldSplitWal && distributedLogReplay) {
-        int timeout = env.getMasterConfiguration().getInt(KEY_WAIT_ON_RIT, DEFAULT_WAIT_ON_RIT);
-        if (!waitOnRegionToClearRegionsInTransition(am, metaHRI, timeout)) {
-          processed = false;
-        } else {
-          // TODO: Matteo. We BLOCK here but most important thing to be doing at this moment.
-          mwm.splitMetaLog(serverName);
-        }
-      }
+    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    for (HRegionInfo hri: am.getRegionStates().getServerRegionInfoSet(serverName)) {
+      if (!isDefaultMetaRegion(hri)) continue;
+
+      am.offlineRegion(hri);
+      addChildProcedure(am.createAssignProcedure(hri, true));
     }
-    return processed;
   }
 
-  /**
-   * @return True if region cleared RIT, else false if we timed out waiting.
-   * @throws InterruptedIOException
-   */
-  private boolean waitOnRegionToClearRegionsInTransition(AssignmentManager am,
-      final HRegionInfo hri, final int timeout)
-  throws InterruptedIOException {
-    try {
-      if (!am.waitOnRegionToClearRegionsInTransition(hri, timeout)) {
-        // Wait here is to avoid log replay hits current dead server and incur a RPC timeout
-        // when replay happens before region assignment completes.
-        LOG.warn("Region " + hri.getEncodedName() + " didn't complete assignment in time");
-        return false;
+  private boolean filterDefaultMetaRegions(final List<HRegionInfo> regions) {
+    if (regions == null) return false;
+    final Iterator<HRegionInfo> it = regions.iterator();
+    while (it.hasNext()) {
+      final HRegionInfo hri = it.next();
+      if (isDefaultMetaRegion(hri)) {
+        it.remove();
       }
-    } catch (InterruptedException ie) {
-      throw new InterruptedIOException("Caught " + ie +
-        " during waitOnRegionToClearRegionsInTransition for " + hri);
     }
-    return true;
+    return !regions.isEmpty();
   }
 
-  private void prepareLogReplay(final MasterProcedureEnv env, final Set<HRegionInfo> regions)
-  throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Mark " + size(this.regionsOnCrashedServer) + " regions-in-recovery from " +
-        this.serverName);
-    }
-    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    mwm.prepareLogReplay(this.serverName, regions);
-    am.getRegionStates().logSplit(this.serverName);
+  private boolean isDefaultMetaRegion(final HRegionInfo hri) {
+    return hri.getTable().equals(TableName.META_TABLE_NAME) &&
+      RegionReplicaUtil.isDefaultReplica(hri);
   }
 
   private void splitLogs(final MasterProcedureEnv env) throws IOException {
@@ -415,124 +283,6 @@ implements ServerProcedureInterface {
     return hris == null? 0: hris.size();
   }
 
-  /**
-   * Figure out what we need to assign. Should be idempotent.
-   * @param env
-   * @return List of calculated regions to assign; may be empty or null.
-   * @throws IOException
-   */
-  private List<HRegionInfo> calcRegionsToAssign(final MasterProcedureEnv env)
-  throws IOException {
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    List<HRegionInfo> regionsToAssignAggregator = new ArrayList<>();
-    int replicaCount = env.getMasterConfiguration().getInt(HConstants.META_REPLICAS_NUM,
-      HConstants.DEFAULT_META_REPLICA_NUM);
-    for (int i = 1; i < replicaCount; i++) {
-      HRegionInfo metaHri =
-          RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, i);
-      if (am.isCarryingMetaReplica(this.serverName, metaHri)) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Reassigning meta replica" + metaHri + " that was on " + this.serverName);
-        }
-        regionsToAssignAggregator.add(metaHri);
-      }
-    }
-    // Clean out anything in regions in transition.
-    List<HRegionInfo> regionsInTransition = am.cleanOutCrashedServerReferences(serverName);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Reassigning " + size(this.regionsOnCrashedServer) +
-        " region(s) that " + (serverName == null? "null": serverName)  +
-        " was carrying (and " + regionsInTransition.size() +
-        " regions(s) that were opening on this server)");
-    }
-    regionsToAssignAggregator.addAll(regionsInTransition);
-
-    // Iterate regions that were on this server and figure which of these we need to reassign
-    if (this.regionsOnCrashedServer != null && !this.regionsOnCrashedServer.isEmpty()) {
-      RegionStates regionStates = am.getRegionStates();
-      for (HRegionInfo hri: this.regionsOnCrashedServer) {
-        if (regionsInTransition.contains(hri)) continue;
-        String encodedName = hri.getEncodedName();
-        Lock lock = am.acquireRegionLock(encodedName);
-        try {
-          RegionState rit = regionStates.getRegionTransitionState(hri);
-          if (processDeadRegion(hri, am)) {
-            ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
-            if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
-              // If this region is in transition on the dead server, it must be
-              // opening or pending_open, which should have been covered by
-              // AM#cleanOutCrashedServerReferences
-              LOG.info("Skip assigning " + hri.getRegionNameAsString()
-                + " because opened on " + addressFromAM.getServerName());
-              continue;
-            }
-            if (rit != null) {
-              if (rit.getServerName() != null && !rit.isOnServer(this.serverName)) {
-                // Skip regions that are in transition on other server
-                LOG.info("Skip assigning region in transition on other server" + rit);
-                continue;
-              }
-              LOG.info("Reassigning region " + rit + " and clearing zknode if exists");
-              regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
-            } else if (regionStates.isRegionInState(
-                hri, RegionState.State.SPLITTING_NEW, RegionState.State.MERGING_NEW)) {
-              regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
-            }
-            regionsToAssignAggregator.add(hri);
-          // TODO: The below else if is different in branch-1 from master branch.
-          } else if (rit != null) {
-            if ((rit.isClosing() || rit.isFailedClose() || rit.isOffline())
-                && am.getTableStateManager().isTableState(hri.getTable(),
-                TableState.State.DISABLED, TableState.State.DISABLING) ||
-                am.getReplicasToClose().contains(hri)) {
-              // If the table was partially disabled and the RS went down, we should clear the
-              // RIT and remove the node for the region.
-              // The rit that we use may be stale in case the table was in DISABLING state
-              // but though we did assign we will not be clearing the znode in CLOSING state.
-              // Doing this will have no harm. See HBASE-5927
-              regionStates.updateRegionState(hri, RegionState.State.OFFLINE);
-              am.offlineDisabledRegion(hri);
-            } else {
-              LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "
-                + rit + " not to be assigned by SSH of server " + serverName);
-            }
-          }
-        } finally {
-          lock.unlock();
-        }
-      }
-    }
-    return regionsToAssignAggregator;
-  }
-
-  private boolean assign(final MasterProcedureEnv env, final List<HRegionInfo> hris)
-  throws InterruptedIOException {
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    try {
-      am.assign(hris);
-    } catch (InterruptedException ie) {
-      LOG.error("Caught " + ie + " during round-robin assignment");
-      throw (InterruptedIOException)new InterruptedIOException().initCause(ie);
-    } catch (IOException ioe) {
-      LOG.info("Caught " + ioe + " during region assignment, will retry");
-      return false;
-    }
-    return true;
-  }
-
-  private boolean waitOnAssign(final MasterProcedureEnv env, final List<HRegionInfo> hris)
-  throws InterruptedIOException {
-    int timeout = env.getMasterConfiguration().getInt(KEY_WAIT_ON_RIT, DEFAULT_WAIT_ON_RIT);
-    for (HRegionInfo hri: hris) {
-      // TODO: Blocks here.
-      if (!waitOnRegionToClearRegionsInTransition(env.getMasterServices().getAssignmentManager(),
-          hri, timeout)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   @Override
   protected void rollbackState(MasterProcedureEnv env, ServerCrashState state)
   throws IOException {
@@ -581,7 +331,7 @@ implements ServerProcedureInterface {
   public void toStringClassDetails(StringBuilder sb) {
     sb.append(getClass().getSimpleName());
     sb.append(" serverName=");
-    sb.append(this.serverName);
+    sb.append(serverName);
     sb.append(", shouldSplitWal=");
     sb.append(shouldSplitWal);
     sb.append(", carryingMeta=");
@@ -595,7 +345,6 @@ implements ServerProcedureInterface {
     MasterProcedureProtos.ServerCrashStateData.Builder state =
       MasterProcedureProtos.ServerCrashStateData.newBuilder().
       setServerName(ProtobufUtil.toServerName(this.serverName)).
-      setDistributedLogReplay(this.distributedLogReplay).
       setCarryingMeta(this.carryingMeta).
       setShouldSplitWal(this.shouldSplitWal);
     if (this.regionsOnCrashedServer != null && !this.regionsOnCrashedServer.isEmpty()) {
@@ -603,11 +352,6 @@ implements ServerProcedureInterface {
         state.addRegionsOnCrashedServer(HRegionInfo.convert(hri));
       }
     }
-    if (this.regionsAssigned != null && !this.regionsAssigned.isEmpty()) {
-      for (HRegionInfo hri: this.regionsAssigned) {
-        state.addRegionsAssigned(HRegionInfo.convert(hri));
-      }
-    }
     state.build().writeDelimitedTo(stream);
   }
 
@@ -618,142 +362,16 @@ implements ServerProcedureInterface {
     MasterProcedureProtos.ServerCrashStateData state =
       MasterProcedureProtos.ServerCrashStateData.parseDelimitedFrom(stream);
     this.serverName = ProtobufUtil.toServerName(state.getServerName());
-    this.distributedLogReplay = state.hasDistributedLogReplay()?
-      state.getDistributedLogReplay(): false;
     this.carryingMeta = state.hasCarryingMeta()? state.getCarryingMeta(): false;
     // shouldSplitWAL has a default over in pb so this invocation will always work.
     this.shouldSplitWal = state.getShouldSplitWal();
     int size = state.getRegionsOnCrashedServerCount();
     if (size > 0) {
-      this.regionsOnCrashedServer = new HashSet<>(size);
+      this.regionsOnCrashedServer = new ArrayList<HRegionInfo>(size);
       for (RegionInfo ri: state.getRegionsOnCrashedServerList()) {
         this.regionsOnCrashedServer.add(HRegionInfo.convert(ri));
       }
     }
-    size = state.getRegionsAssignedCount();
-    if (size > 0) {
-      this.regionsAssigned = new ArrayList<>(size);
-      for (RegionInfo ri: state.getRegionsOnCrashedServerList()) {
-        this.regionsAssigned.add(HRegionInfo.convert(ri));
-      }
-    }
-  }
-
-  /**
-   * Process a dead region from a dead RS. Checks if the region is disabled or
-   * disabling or if the region has a partially completed split.
-   * @param hri
-   * @param assignmentManager
-   * @return Returns true if specified region should be assigned, false if not.
-   * @throws IOException
-   */
-  private static boolean processDeadRegion(HRegionInfo hri, AssignmentManager assignmentManager)
-  throws IOException {
-    boolean tablePresent = assignmentManager.getTableStateManager().isTablePresent(hri.getTable());
-    if (!tablePresent) {
-      LOG.info("The table " + hri.getTable() + " was deleted.  Hence not proceeding.");
-      return false;
-    }
-    // If table is not disabled but the region is offlined,
-    boolean disabled = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
-      TableState.State.DISABLED);
-    if (disabled){
-      LOG.info("The table " + hri.getTable() + " was disabled.  Hence not proceeding.");
-      return false;
-    }
-    if (hri.isOffline() && hri.isSplit()) {
-      // HBASE-7721: Split parent and daughters are inserted into hbase:meta as an atomic operation.
-      // If the meta scanner saw the parent split, then it should see the daughters as assigned
-      // to the dead server. We don't have to do anything.
-      return false;
-    }
-    boolean disabling = assignmentManager.getTableStateManager().isTableState(hri.getTable(),
-        TableState.State.DISABLING);
-    if (disabling) {
-      LOG.info("The table " + hri.getTable() + " is disabled.  Hence not assigning region" +
-        hri.getEncodedName());
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * If hbase:meta is not assigned already, assign.
-   * @throws IOException
-   */
-  private void verifyAndAssignMetaWithRetries(final MasterProcedureEnv env) throws IOException {
-    MasterServices services = env.getMasterServices();
-    int iTimes = services.getConfiguration().getInt(KEY_RETRIES_ON_META, DEFAULT_RETRIES_ON_META);
-    // Just reuse same time as we have for short wait on meta. Adding another config is overkill.
-    long waitTime =
-      services.getConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META);
-    int iFlag = 0;
-    while (true) {
-      try {
-        verifyAndAssignMeta(env);
-        break;
-      } catch (KeeperException e) {
-        services.abort("In server shutdown processing, assigning meta", e);
-        throw new IOException("Aborting", e);
-      } catch (Exception e) {
-        if (iFlag >= iTimes) {
-          services.abort("verifyAndAssignMeta failed after" + iTimes + " retries, aborting", e);
-          throw new IOException("Aborting", e);
-        }
-        try {
-          Thread.sleep(waitTime);
-        } catch (InterruptedException e1) {
-          LOG.warn("Interrupted when is the thread sleep", e1);
-          Thread.currentThread().interrupt();
-          throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
-        }
-        iFlag++;
-      }
-    }
-  }
-
-  /**
-   * If hbase:meta is not assigned already, assign.
-   * @throws InterruptedException
-   * @throws IOException
-   * @throws KeeperException
-   */
-  private void verifyAndAssignMeta(final MasterProcedureEnv env)
-      throws InterruptedException, IOException, KeeperException {
-    MasterServices services = env.getMasterServices();
-    if (!isMetaAssignedQuickTest(env)) {
-      services.getAssignmentManager().assignMeta(HRegionInfo.FIRST_META_REGIONINFO);
-    } else if (serverName.equals(services.getMetaTableLocator().
-        getMetaRegionLocation(services.getZooKeeper()))) {
-      throw new IOException("hbase:meta is onlined on the dead server " + this.serverName);
-    } else {
-      LOG.info("Skip assigning hbase:meta because it is online at "
-          + services.getMetaTableLocator().getMetaRegionLocation(services.getZooKeeper()));
-    }
-  }
-
-  /**
-   * A quick test that hbase:meta is assigned; blocks for short time only.
-   * @return True if hbase:meta location is available and verified as good.
-   * @throws InterruptedException
-   * @throws IOException
-   */
-  private boolean isMetaAssignedQuickTest(final MasterProcedureEnv env)
-  throws InterruptedException, IOException {
-    ZooKeeperWatcher zkw = env.getMasterServices().getZooKeeper();
-    MetaTableLocator mtl = env.getMasterServices().getMetaTableLocator();
-    boolean metaAssigned = false;
-    // Is hbase:meta location available yet?
-    if (mtl.isLocationAvailable(zkw)) {
-      ClusterConnection connection = env.getMasterServices().getClusterConnection();
-      // Is hbase:meta location good yet?
-      long timeout =
-        env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META);
-      if (mtl.verifyMetaRegionLocation(connection, zkw, timeout)) {
-        metaAssigned = true;
-      }
-    }
-    return metaAssigned;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
deleted file mode 100644
index 3cd6c66..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
+++ /dev/null
@@ -1,785 +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 java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.Threads;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * The procedure to split a region in a table.
- */
-@InterfaceAudience.Private
-public class SplitTableRegionProcedure
-    extends AbstractStateMachineTableProcedure<SplitTableRegionState> {
-  private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
-
-  private Boolean traceEnabled;
-
-  /*
-   * Region to split
-   */
-  private HRegionInfo parentHRI;
-  private HRegionInfo daughter_1_HRI;
-  private HRegionInfo daughter_2_HRI;
-
-  public SplitTableRegionProcedure() {
-    this.traceEnabled = null;
-  }
-
-  public SplitTableRegionProcedure(final MasterProcedureEnv env,
-      final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
-    super(env);
-
-    checkSplitRow(regionToSplit, splitRow);
-
-    this.traceEnabled = null;
-    this.parentHRI = regionToSplit;
-
-    final TableName table = regionToSplit.getTable();
-    final long rid = getDaughterRegionIdTimestamp(regionToSplit);
-    this.daughter_1_HRI = new HRegionInfo(table, regionToSplit.getStartKey(), splitRow, false, rid);
-    this.daughter_2_HRI = new HRegionInfo(table, splitRow, regionToSplit.getEndKey(), false, rid);
-  }
-
-  private static void checkSplitRow(final HRegionInfo regionToSplit, final byte[] splitRow)
-      throws IOException {
-    if (splitRow == null || splitRow.length == 0) {
-      throw new DoNotRetryIOException("Split row cannot be null");
-    }
-
-    if (Bytes.equals(regionToSplit.getStartKey(), splitRow)) {
-      throw new DoNotRetryIOException(
-        "Split row is equal to startkey: " + Bytes.toStringBinary(splitRow));
-    }
-
-    if (!regionToSplit.containsRow(splitRow)) {
-      throw new DoNotRetryIOException(
-        "Split row is not inside region key range splitKey:" + Bytes.toStringBinary(splitRow) +
-        " region: " + regionToSplit);
-    }
-  }
-
-  /**
-   * Calculate daughter regionid to use.
-   * @param hri Parent {@link HRegionInfo}
-   * @return Daughter region id (timestamp) to use.
-   */
-  private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
-    long rid = EnvironmentEdgeManager.currentTime();
-    // Regionid is timestamp.  Can't be less than that of parent else will insert
-    // at wrong location in hbase:meta (See HBASE-710).
-    if (rid < hri.getRegionId()) {
-      LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
-        " but current time here is " + rid);
-      rid = hri.getRegionId() + 1;
-    }
-    return rid;
-  }
-
-  @Override
-  protected Flow executeFromState(final MasterProcedureEnv env, final SplitTableRegionState state)
-      throws InterruptedException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-
-    try {
-      switch (state) {
-      case SPLIT_TABLE_REGION_PREPARE:
-        if (prepareSplitRegion(env)) {
-          setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION);
-          break;
-        } else {
-          assert isFailed() : "split region should have an exception here";
-          return Flow.NO_MORE_STATE;
-        }
-      case SPLIT_TABLE_REGION_PRE_OPERATION:
-        preSplitRegion(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE);
-        break;
-      case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
-        setRegionStateToSplitting(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION);
-        break;
-      case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
-        closeParentRegionForSplit(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
-        break;
-      case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
-        createDaughterRegions(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR);
-        break;
-      case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
-        preSplitRegionBeforePONR(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_UPDATE_META);
-        break;
-      case SPLIT_TABLE_REGION_UPDATE_META:
-        // This is the point of no return.  Adding subsequent edits to .META. as we
-        // do below when we do the daughter opens adding each to .META. can fail in
-        // various interesting ways the most interesting of which is a timeout
-        // BUT the edits all go through (See HBASE-3872).  IF we reach the PONR
-        // then subsequent failures need to crash out this region server; the
-        // server shutdown processing should be able to fix-up the incomplete split.
-        // The offlined parent will have the daughters as extra columns.  If
-        // we leave the daughter regions in place and do not remove them when we
-        // crash out, then they will have their references to the parent in place
-        // still and the server shutdown fixup of .META. will point to these
-        // regions.
-        // We should add PONR JournalEntry before offlineParentInMeta,so even if
-        // OfflineParentInMeta timeout,this will cause regionserver exit,and then
-        // master ServerShutdownHandler will fix daughter & avoid data loss. (See
-        // HBase-4562).
-        updateMetaForDaughterRegions(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR);
-        break;
-      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
-        preSplitRegionAfterPONR(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS);
-        break;
-      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
-        openDaughterRegions(env);
-        setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_POST_OPERATION);
-        break;
-      case SPLIT_TABLE_REGION_POST_OPERATION:
-        postSplitRegion(env);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      String msg = "Error trying to split region " + parentHRI.getEncodedName() + " in the table "
-          + getTableName() + " (in state=" + state + ")";
-      if (!isRollbackSupported(state)) {
-        // We reach a state that cannot be rolled back. We just need to keep retry.
-        LOG.warn(msg, e);
-      } else {
-        LOG.error(msg, e);
-        setFailure("master-split-region", e);
-      }
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  @Override
-  protected void rollbackState(final MasterProcedureEnv env, final SplitTableRegionState state)
-      throws IOException, InterruptedException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " rollback state=" + state);
-    }
-
-    try {
-      switch (state) {
-      case SPLIT_TABLE_REGION_POST_OPERATION:
-      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
-      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
-      case SPLIT_TABLE_REGION_UPDATE_META:
-        // PONR
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR:
-        break;
-      case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
-        // Doing nothing, as re-open parent region would clean up daughter region directories.
-        break;
-      case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
-        openParentRegion(env);
-        break;
-      case SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE:
-        setRegionStateToRevertSplitting(env);
-        break;
-      case SPLIT_TABLE_REGION_PRE_OPERATION:
-        postRollBackSplitRegion(env);
-        break;
-      case SPLIT_TABLE_REGION_PREPARE:
-        break; // nothing to do
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      // This will be retried. Unless there is a bug in the code,
-      // this should be just a "temporary error" (e.g. network down)
-      LOG.warn("Failed rollback attempt step " + state + " for splitting the region "
-        + parentHRI.getEncodedName() + " in table " + getTableName(), e);
-      throw e;
-    }
-  }
-
-  /*
-   * Check whether we are in the state that can be rollback
-   */
-  @Override
-  protected boolean isRollbackSupported(final SplitTableRegionState state) {
-    switch (state) {
-      case SPLIT_TABLE_REGION_POST_OPERATION:
-      case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
-      case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR:
-      case SPLIT_TABLE_REGION_UPDATE_META:
-        // It is not safe to rollback if we reach to these states.
-        return false;
-      default:
-        break;
-    }
-    return true;
-  }
-
-  @Override
-  protected SplitTableRegionState getState(final int stateId) {
-    return SplitTableRegionState.valueOf(stateId);
-  }
-
-  @Override
-  protected int getStateId(final SplitTableRegionState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected SplitTableRegionState getInitialState() {
-    return SplitTableRegionState.SPLIT_TABLE_REGION_PREPARE;
-  }
-
-  @Override
-  public void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-
-    final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
-        MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
-        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
-        .setParentRegionInfo(HRegionInfo.convert(parentHRI))
-        .addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
-        .addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
-    splitTableRegionMsg.build().writeDelimitedTo(stream);
-  }
-
-  @Override
-  public void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-
-    final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
-        MasterProcedureProtos.SplitTableRegionStateData.parseDelimitedFrom(stream);
-    setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
-    parentHRI = HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo());
-    if (splitTableRegionsMsg.getChildRegionInfoCount() == 0) {
-      daughter_1_HRI = daughter_2_HRI = null;
-    } else {
-      assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
-      daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfoList().get(0));
-      daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfoList().get(1));
-    }
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" (table=");
-    sb.append(getTableName());
-    sb.append(" parent region=");
-    sb.append(parentHRI);
-    if (daughter_1_HRI != null) {
-      sb.append(" first daughter region=");
-      sb.append(daughter_1_HRI);
-    }
-    if (daughter_2_HRI != null) {
-      sb.append(" and second daughter region=");
-      sb.append(daughter_2_HRI);
-    }
-    sb.append(")");
-  }
-
-  @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    return env.getProcedureScheduler().waitRegions(this, getTableName(), parentHRI)?
-        LockState.LOCK_EVENT_WAIT: LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegions(this, getTableName(), parentHRI);
-  }
-
-  @Override
-  public TableName getTableName() {
-    return parentHRI.getTable();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.SPLIT;
-  }
-
-  private byte[] getSplitRow() {
-    return daughter_2_HRI.getStartKey();
-  }
-
-  /**
-   * Prepare to Split region.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  @VisibleForTesting
-  public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
-    // Check whether the region is splittable
-    final RegionState state = getParentRegionState(env);
-    if (state.isClosing() || state.isClosed() ||
-        state.isSplittingOrSplitOnServer(state.getServerName())) {
-      setFailure(
-        "master-split-region",
-        new IOException("Split region " + parentHRI + " failed due to region is not splittable"));
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Action before splitting region in a table.
-   * @param env MasterProcedureEnv
-   * @param state the procedure state
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private void preSplitRegion(final MasterProcedureEnv env)
-      throws IOException, InterruptedException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.preSplitRegionAction(getTableName(), getSplitRow(), getUser());
-    }
-  }
-
-  /**
-   * Action after rollback a split table region action.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void postRollBackSplitRegion(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.postRollBackSplitRegionAction(getUser());
-    }
-  }
-
-  /**
-   * Set the parent region state to SPLITTING state
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  @VisibleForTesting
-  public void setRegionStateToSplitting(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.READY_TO_SPLIT);
-    transition.addRegionInfo(HRegionInfo.convert(parentHRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
-    if (env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getParentRegionState(env).getServerName(), transition.build()) != null) {
-      throw new IOException("Failed to update region state to SPLITTING for "
-          + parentHRI.getRegionNameAsString());
-    }
-  }
-
-  /**
-   * Rollback the region state change
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void setRegionStateToRevertSplitting(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.SPLIT_REVERTED);
-    transition.addRegionInfo(HRegionInfo.convert(parentHRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
-    if (env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getParentRegionState(env).getServerName(), transition.build()) != null) {
-      throw new IOException("Failed to update region state for "
-          + parentHRI.getRegionNameAsString() + " as part of operation for reverting split");
-    }
-  }
-
-  /**
-   * RPC to region server that host the parent region, ask for close the parent regions
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  @VisibleForTesting
-  public void closeParentRegionForSplit(final MasterProcedureEnv env) throws IOException {
-    boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplitOrMerge(
-      getParentRegionState(env).getServerName(), parentHRI);
-    if (!success) {
-      throw new IOException("Close parent region " + parentHRI + " for splitting failed."
-        + "  Check region server log for more details");
-    }
-  }
-
-  /**
-   * Rollback close parent region
-   * @param env MasterProcedureEnv
-   **/
-  private void openParentRegion(final MasterProcedureEnv env) throws IOException {
-    // Check whether the region is closed; if so, open it in the same server
-    RegionState state = getParentRegionState(env);
-    if (state.isClosing() || state.isClosed()) {
-      env.getMasterServices().getServerManager().sendRegionOpen(
-        getParentRegionState(env).getServerName(),
-        parentHRI,
-        ServerName.EMPTY_SERVER_LIST);
-    }
-  }
-
-  /**
-   * Create daughter regions
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  @VisibleForTesting
-  public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), parentHRI.getTable());
-    final FileSystem fs = mfs.getFileSystem();
-    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, parentHRI, false);
-    regionFs.createSplitsDir();
-
-    Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
-
-    assertReferenceFileCount(
-      fs, expectedReferences.getFirst(), regionFs.getSplitsDir(daughter_1_HRI));
-    //Move the files from the temporary .splits to the final /table/region directory
-    regionFs.commitDaughterRegion(daughter_1_HRI);
-    assertReferenceFileCount(
-      fs,
-      expectedReferences.getFirst(),
-      new Path(tabledir, daughter_1_HRI.getEncodedName()));
-
-    assertReferenceFileCount(
-      fs, expectedReferences.getSecond(), regionFs.getSplitsDir(daughter_2_HRI));
-    regionFs.commitDaughterRegion(daughter_2_HRI);
-    assertReferenceFileCount(
-      fs,
-      expectedReferences.getSecond(),
-      new Path(tabledir, daughter_2_HRI.getEncodedName()));
-  }
-
-  /**
-   * Create Split directory
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private Pair<Integer, Integer> splitStoreFiles(final MasterProcedureEnv env,
-      final HRegionFileSystem regionFs) throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Configuration conf = env.getMasterConfiguration();
-
-    // The following code sets up a thread pool executor with as many slots as
-    // there's files to split. It then fires up everything, waits for
-    // completion and finally checks for any exception
-    //
-    // Note: splitStoreFiles creates daughter region dirs under the parent splits dir
-    // Nothing to unroll here if failure -- re-run createSplitsDir will
-    // clean this up.
-    int nbFiles = 0;
-    for (String family: regionFs.getFamilies()) {
-      Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
-      if (storeFiles != null) {
-        nbFiles += storeFiles.size();
-      }
-    }
-    if (nbFiles == 0) {
-      // no file needs to be splitted.
-      return new Pair<>(0,0);
-    }
-    // Default max #threads to use is the smaller of table's configured number of blocking store
-    // files or the available number of logical cores.
-    int defMaxThreads = Math.min(
-      conf.getInt(HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT),
-      Runtime.getRuntime().availableProcessors());
-    // Max #threads is the smaller of the number of storefiles or the default max determined above.
-    int maxThreads = Math.min(
-      conf.getInt(HConstants.REGION_SPLIT_THREADS_MAX, defMaxThreads), nbFiles);
-    LOG.info("Preparing to split " + nbFiles + " storefiles for region " + parentHRI +
-            " using " + maxThreads + " threads");
-    ThreadPoolExecutor threadPool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
-      maxThreads, Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
-    List<Future<Pair<Path,Path>>> futures = new ArrayList<>(nbFiles);
-
-    // Split each store file.
-    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
-    for (String family: regionFs.getFamilies()) {
-      final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
-      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
-      if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          StoreFileSplitter sfs = new StoreFileSplitter(regionFs, family.getBytes(),
-            new StoreFile(mfs.getFileSystem(), storeFileInfo, conf,
-              cacheConf, hcd.getBloomFilterType()));
-          futures.add(threadPool.submit(sfs));
-        }
-      }
-    }
-    // Shutdown the pool
-    threadPool.shutdown();
-
-    // Wait for all the tasks to finish
-    long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 30000);
-    try {
-      boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS);
-      if (stillRunning) {
-        threadPool.shutdownNow();
-        // wait for the thread to shutdown completely.
-        while (!threadPool.isTerminated()) {
-          Thread.sleep(50);
-        }
-        throw new IOException("Took too long to split the" +
-            " files and create the references, aborting split");
-      }
-    } catch (InterruptedException e) {
-      throw (InterruptedIOException)new InterruptedIOException().initCause(e);
-    }
-
-    int daughterA = 0;
-    int daughterB = 0;
-    // Look for any exception
-    for (Future<Pair<Path, Path>> future : futures) {
-      try {
-        Pair<Path, Path> p = future.get();
-        daughterA += p.getFirst() != null ? 1 : 0;
-        daughterB += p.getSecond() != null ? 1 : 0;
-      } catch (InterruptedException e) {
-        throw (InterruptedIOException) new InterruptedIOException().initCause(e);
-      } catch (ExecutionException e) {
-        throw new IOException(e);
-      }
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Split storefiles for region " + parentHRI + " Daughter A: " + daughterA
-          + " storefiles, Daughter B: " + daughterB + " storefiles.");
-    }
-    return new Pair<>(daughterA, daughterB);
-  }
-
-  private void assertReferenceFileCount(
-      final FileSystem fs,
-      final int expectedReferenceFileCount,
-      final Path dir)
-      throws IOException {
-    if (expectedReferenceFileCount != 0 &&
-        expectedReferenceFileCount != FSUtils.getRegionReferenceFileCount(fs, dir)) {
-      throw new IOException("Failing split. Expected reference file count isn't equal.");
-    }
-  }
-
-  private Pair<Path, Path> splitStoreFile(final HRegionFileSystem regionFs,
-      final byte[] family, final StoreFile sf) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Splitting started for store file: " + sf.getPath() + " for region: " + parentHRI);
-    }
-
-    final byte[] splitRow = getSplitRow();
-    final String familyName = Bytes.toString(family);
-    final Path path_first =
-        regionFs.splitStoreFile(this.daughter_1_HRI, familyName, sf, splitRow, false, null);
-    final Path path_second =
-        regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, splitRow, true, null);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Splitting complete for store file: " + sf.getPath() + " for region: " + parentHRI);
-    }
-    return new Pair<>(path_first, path_second);
-  }
-
-  /**
-   * Utility class used to do the file splitting / reference writing
-   * in parallel instead of sequentially.
-   */
-  private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
-    private final HRegionFileSystem regionFs;
-    private final byte[] family;
-    private final StoreFile sf;
-
-    /**
-     * Constructor that takes what it needs to split
-     * @param regionFs the file system
-     * @param family Family that contains the store file
-     * @param sf which file
-     */
-    public StoreFileSplitter(
-        final HRegionFileSystem regionFs,
-        final byte[] family,
-        final StoreFile sf) {
-      this.regionFs = regionFs;
-      this.sf = sf;
-      this.family = family;
-    }
-
-    public Pair<Path,Path> call() throws IOException {
-      return splitStoreFile(regionFs, family, sf);
-    }
-  }
-
-  /**
-   * Post split region actions before the Point-of-No-Return step
-   * @param env MasterProcedureEnv
-   **/
-  private void preSplitRegionBeforePONR(final MasterProcedureEnv env)
-    throws IOException, InterruptedException {
-    final List<Mutation> metaEntries = new ArrayList<>();
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      if (cpHost.preSplitBeforePONRAction(getSplitRow(), metaEntries, getUser())) {
-        throw new IOException("Coprocessor bypassing region " +
-            parentHRI.getRegionNameAsString() + " split.");
-      }
-      try {
-        for (Mutation p : metaEntries) {
-          HRegionInfo.parseRegionName(p.getRow());
-        }
-      } catch (IOException e) {
-        LOG.error("Row key of mutation from coprocessor is not parsable as region name."
-            + "Mutations from coprocessor should only for hbase:meta table.");
-        throw e;
-      }
-    }
-  }
-
-  /**
-   * Add daughter regions to META
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.SPLIT_PONR);
-    transition.addRegionInfo(HRegionInfo.convert(parentHRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_1_HRI));
-    transition.addRegionInfo(HRegionInfo.convert(daughter_2_HRI));
-    if (env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getParentRegionState(env).getServerName(), transition.build()) != null) {
-      throw new IOException("Failed to update meta to add daughter regions in split region "
-          + parentHRI.getRegionNameAsString());
-    }
-  }
-
-  /**
-   * Pre split region actions after the Point-of-No-Return step
-   * @param env MasterProcedureEnv
-   **/
-  private void preSplitRegionAfterPONR(final MasterProcedureEnv env)
-      throws IOException, InterruptedException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.preSplitAfterPONRAction(getUser());
-    }
-  }
-
-  /**
-   * Assign daughter regions
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   * @throws InterruptedException
-   **/
-  private void openDaughterRegions(final MasterProcedureEnv env)
-      throws IOException, InterruptedException {
-    env.getMasterServices().getAssignmentManager().assignDaughterRegions(
-      parentHRI, daughter_1_HRI, daughter_2_HRI);
-  }
-
-  /**
-   * Post split region actions
-   * @param env MasterProcedureEnv
-   **/
-  private void postSplitRegion(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.postCompletedSplitRegionAction(daughter_1_HRI, daughter_2_HRI, getUser());
-    }
-  }
-
-  /**
-   * Get parent region state
-   * @param env MasterProcedureEnv
-   * @return parent region state
-   */
-  private RegionState getParentRegionState(final MasterProcedureEnv env) {
-    RegionStates regionStates = env.getMasterServices().getAssignmentManager().getRegionStates();
-    RegionState state = regionStates.getRegionState(parentHRI);
-    if (state == null) {
-      LOG.warn("Split but not in region states: " + parentHRI);
-      state = regionStates.createRegionState(parentHRI);
-    }
-    return state;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return traceEnabled
-   */
-  private boolean isTraceEnabled() {
-    if (traceEnabled == null) {
-      traceEnabled = LOG.isTraceEnabled();
-    }
-    return traceEnabled;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 2ab142a..86a9acf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -85,7 +85,7 @@ public class TruncateTableProcedure
 
           // TODO: Move out... in the acquireLock()
           LOG.debug("waiting for '" + getTableName() + "' regions in transition");
-          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+          regions = env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName());
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 
@@ -121,12 +121,14 @@ public class TruncateTableProcedure
           setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
           break;
         case TRUNCATE_TABLE_ASSIGN_REGIONS:
-          CreateTableProcedure.assignRegions(env, getTableName(), regions);
+          CreateTableProcedure.setEnablingState(env, getTableName());
+          addChildProcedure(env.getAssignmentManager().createAssignProcedures(regions));
           setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
           hTableDescriptor = null;
           regions = null;
           break;
         case TRUNCATE_TABLE_POST_OPERATION:
+          CreateTableProcedure.setEnabledState(env, getTableName());
           postTruncate(env);
           LOG.debug("truncate '" + getTableName() + "' completed");
           return Flow.NO_MORE_STATE;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 5dab2e3..9385a7f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -312,7 +312,7 @@ public class MasterQuotaManager implements RegionStateListener {
       namespaceQuotaManager.checkQuotaToCreateTable(tName, regions);
     }
   }
-  
+
   public void checkAndUpdateNamespaceRegionQuota(TableName tName, int regions) throws IOException {
     if (enabled) {
       namespaceQuotaManager.checkQuotaToUpdateRegion(tName, regions);
@@ -329,18 +329,27 @@ public class MasterQuotaManager implements RegionStateListener {
     return -1;
   }
 
+  @Override
   public void onRegionMerged(HRegionInfo hri) throws IOException {
     if (enabled) {
       namespaceQuotaManager.updateQuotaForRegionMerge(hri);
     }
   }
 
+  @Override
   public void onRegionSplit(HRegionInfo hri) throws IOException {
     if (enabled) {
       namespaceQuotaManager.checkQuotaToSplitRegion(hri);
     }
   }
 
+  @Override
+  public void onRegionSplitReverted(HRegionInfo hri) throws IOException {
+    if (enabled) {
+      this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
+    }
+  }
+
   /**
    * Remove table from namespace quota.
    *
@@ -478,12 +487,5 @@ public class MasterQuotaManager implements RegionStateListener {
       }
     }
   }
-
-  @Override
-  public void onRegionSplitReverted(HRegionInfo hri) throws IOException {
-    if (enabled) {
-      this.namespaceQuotaManager.removeRegionFromNamespaceUsage(hri);
-    }
-  }
 }
 


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
deleted file mode 100644
index 366378a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
+++ /dev/null
@@ -1,909 +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 java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaMutationAnnotation;
-import org.apache.hadoop.hbase.RegionLoad;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.CatalogJanitor;
-import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-
-/**
- * The procedure to Merge a region in a table.
- */
-@InterfaceAudience.Private
-public class MergeTableRegionsProcedure
-    extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
-  private static final Log LOG = LogFactory.getLog(MergeTableRegionsProcedure.class);
-
-  private Boolean traceEnabled;
-  private AssignmentManager assignmentManager;
-  private int timeout;
-  private ServerName regionLocation;
-  private String regionsToMergeListFullName;
-  private String regionsToMergeListEncodedName;
-
-  private HRegionInfo [] regionsToMerge;
-  private HRegionInfo mergedRegionInfo;
-  private boolean forcible;
-
-  public MergeTableRegionsProcedure() {
-    this.traceEnabled = isTraceEnabled();
-    this.assignmentManager = null;
-    this.timeout = -1;
-    this.regionLocation = null;
-    this.regionsToMergeListFullName = null;
-    this.regionsToMergeListEncodedName = null;
-  }
-
-  public MergeTableRegionsProcedure(
-      final MasterProcedureEnv env,
-      final HRegionInfo[] regionsToMerge,
-      final boolean forcible) throws IOException {
-    super(env);
-    this.traceEnabled = isTraceEnabled();
-    this.assignmentManager = getAssignmentManager(env);
-    // For now, we only merge 2 regions.  It could be extended to more than 2 regions in
-    // the future.
-    assert(regionsToMerge.length == 2);
-    assert(regionsToMerge[0].getTable() == regionsToMerge[1].getTable());
-    this.regionsToMerge = regionsToMerge;
-    this.forcible = forcible;
-
-    this.timeout = -1;
-    this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
-    this.regionsToMergeListEncodedName = getRegionsToMergeListEncodedNameString();
-
-    // Check daughter regions and make sure that we have valid daughter regions before
-    // doing the real work.
-    checkDaughterRegions();
-    // WARN: make sure there is no parent region of the two merging regions in
-    // hbase:meta If exists, fixing up daughters would cause daughter regions(we
-    // have merged one) online again when we restart master, so we should clear
-    // the parent region to prevent the above case
-    // Since HBASE-7721, we don't need fix up daughters any more. so here do
-    // nothing
-    setupMergedRegionInfo();
-  }
-
-  @Override
-  protected Flow executeFromState(
-      final MasterProcedureEnv env,
-      final MergeTableRegionsState state) throws InterruptedException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-
-    try {
-      switch (state) {
-      case MERGE_TABLE_REGIONS_PREPARE:
-        prepareMergeRegion(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS);
-        break;
-      case MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS:
-        if (MoveRegionsToSameRS(env)) {
-          setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION);
-        } else {
-          LOG.info("Cancel merging regions " + getRegionsToMergeListFullNameString()
-            + ", because can't move them to the same RS");
-          setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
-        }
-        break;
-      case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
-        preMergeRegions(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE);
-        break;
-      case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
-        setRegionStateToMerging(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CLOSE_REGIONS);
-        break;
-      case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
-        closeRegionsForMerge(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
-        break;
-      case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
-        createMergedRegion(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION);
-        break;
-      case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
-        preMergeRegionsCommit(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_UPDATE_META);
-        break;
-      case MERGE_TABLE_REGIONS_UPDATE_META:
-        updateMetaForMergedRegions(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION);
-        break;
-      case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
-        postMergeRegionsCommit(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_OPEN_MERGED_REGION);
-        break;
-      case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
-        openMergedRegions(env);
-        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
-        break;
-      case MERGE_TABLE_REGIONS_POST_OPERATION:
-        postCompletedMergeRegions(env);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (IOException e) {
-      LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
-        " in the table " + getTableName() + " (in state=" + state + ")", e);
-
-      setFailure("master-merge-regions", e);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  @Override
-  protected void rollbackState(
-      final MasterProcedureEnv env,
-      final MergeTableRegionsState state) throws IOException, InterruptedException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " rollback state=" + state);
-    }
-
-    try {
-      switch (state) {
-      case MERGE_TABLE_REGIONS_POST_OPERATION:
-      case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
-      case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
-      case MERGE_TABLE_REGIONS_UPDATE_META:
-        String msg = this + " We are in the " + state + " state."
-            + " It is complicated to rollback the merge operation that region server is working on."
-            + " Rollback is not supported and we should let the merge operation to complete";
-        LOG.warn(msg);
-        // PONR
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
-        break;
-      case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
-        cleanupMergedRegion(env);
-        break;
-      case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
-        rollbackCloseRegionsForMerge(env);
-        break;
-      case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
-        setRegionStateToRevertMerging(env);
-        break;
-      case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
-        postRollBackMergeRegions(env);
-        break;
-      case MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS:
-        break; // nothing to rollback
-      case MERGE_TABLE_REGIONS_PREPARE:
-        break; // nothing to rollback
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (Exception e) {
-      // This will be retried. Unless there is a bug in the code,
-      // this should be just a "temporary error" (e.g. network down)
-      LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
-          + getRegionsToMergeListFullNameString() + " in table " + getTableName(), e);
-      throw e;
-    }
-  }
-
-  /*
-   * Check whether we are in the state that can be rollback
-   */
-  @Override
-  protected boolean isRollbackSupported(final MergeTableRegionsState state) {
-    switch (state) {
-    case MERGE_TABLE_REGIONS_POST_OPERATION:
-    case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
-    case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
-    case MERGE_TABLE_REGIONS_UPDATE_META:
-        // It is not safe to rollback if we reach to these states.
-        return false;
-      default:
-        break;
-    }
-    return true;
-  }
-
-  @Override
-  protected MergeTableRegionsState getState(final int stateId) {
-    return MergeTableRegionsState.valueOf(stateId);
-  }
-
-  @Override
-  protected int getStateId(final MergeTableRegionsState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected MergeTableRegionsState getInitialState() {
-    return MergeTableRegionsState.MERGE_TABLE_REGIONS_PREPARE;
-  }
-
-  @Override
-  public void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-
-    MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
-        MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
-        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
-        .setMergedRegionInfo(HRegionInfo.convert(mergedRegionInfo))
-        .setForcible(forcible);
-    for (HRegionInfo hri: regionsToMerge) {
-      mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(hri));
-    }
-    mergeTableRegionsMsg.build().writeDelimitedTo(stream);
-  }
-
-  @Override
-  public void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-
-    MasterProcedureProtos.MergeTableRegionsStateData mergeTableRegionsMsg =
-        MasterProcedureProtos.MergeTableRegionsStateData.parseDelimitedFrom(stream);
-    setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
-
-    assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
-    regionsToMerge = new HRegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
-    for (int i = 0; i < regionsToMerge.length; i++) {
-      regionsToMerge[i] = HRegionInfo.convert(mergeTableRegionsMsg.getRegionInfo(i));
-    }
-
-    mergedRegionInfo = HRegionInfo.convert(mergeTableRegionsMsg.getMergedRegionInfo());
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" (table=");
-    sb.append(getTableName());
-    sb.append(" regions=");
-    sb.append(getRegionsToMergeListFullNameString());
-    sb.append(" forcible=");
-    sb.append(forcible);
-    sb.append(")");
-  }
-
-  @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) {
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    return env.getProcedureScheduler().waitRegions(this, getTableName(),
-        regionsToMerge[0], regionsToMerge[1])?
-            LockState.LOCK_EVENT_WAIT: LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegions(this, getTableName(),
-        regionsToMerge[0], regionsToMerge[1]);
-  }
-
-  @Override
-  public TableName getTableName() {
-    return regionsToMerge[0].getTable();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.MERGE;
-  }
-
-  /**
-   * check daughter regions
-   * @throws IOException
-   */
-  private void checkDaughterRegions() throws IOException {
-    // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
-    // if we want to extend to more than 2 regions, the code needs to modify a little bit.
-    //
-    if (regionsToMerge[0].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
-        regionsToMerge[1].getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
-      throw new MergeRegionException("Can't merge non-default replicas");
-    }
-
-    if (!HRegionInfo.areAdjacent(regionsToMerge[0], regionsToMerge[1])) {
-      String msg = "Trying to merge non-adjacent regions "
-          + getRegionsToMergeListFullNameString() + " where forcible = " + forcible;
-      LOG.warn(msg);
-      if (!forcible) {
-        throw new DoNotRetryIOException(msg);
-      }
-    }
-  }
-
-  /**
-   * Prepare merge and do some check
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
-    // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
-    // if we want to extend to more than 2 regions, the code needs to modify a little bit.
-    //
-    CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
-    boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
-    if (regionAHasMergeQualifier
-        || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
-      String msg = "Skip merging regions " + getRegionsToMergeListFullNameString()
-        + ", because region "
-        + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
-              .getEncodedName()) + " has merge qualifier";
-      LOG.warn(msg);
-      throw new MergeRegionException(msg);
-    }
-
-    RegionStates regionStates = getAssignmentManager(env).getRegionStates();
-    RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
-    RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
-    if (regionStateA == null || regionStateB == null) {
-      throw new UnknownRegionException(
-        regionStateA == null ?
-            regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
-    }
-
-    if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
-      throw new MergeRegionException(
-        "Unable to merge regions not online " + regionStateA + ", " + regionStateB);
-    }
-  }
-
-  /**
-   * Create merged region info through the specified two regions
-   */
-  private void setupMergedRegionInfo() {
-    long rid = EnvironmentEdgeManager.currentTime();
-    // Regionid is timestamp. Merged region's id can't be less than that of
-    // merging regions else will insert at wrong location in hbase:meta
-    if (rid < regionsToMerge[0].getRegionId() || rid < regionsToMerge[1].getRegionId()) {
-      LOG.warn("Clock skew; merging regions id are " + regionsToMerge[0].getRegionId()
-          + " and " + regionsToMerge[1].getRegionId() + ", but current time here is " + rid);
-      rid = Math.max(regionsToMerge[0].getRegionId(), regionsToMerge[1].getRegionId()) + 1;
-    }
-
-    byte[] startKey = null;
-    byte[] endKey = null;
-    // Choose the smaller as start key
-    if (regionsToMerge[0].compareTo(regionsToMerge[1]) <= 0) {
-      startKey = regionsToMerge[0].getStartKey();
-    } else {
-      startKey = regionsToMerge[1].getStartKey();
-    }
-    // Choose the bigger as end key
-    if (Bytes.equals(regionsToMerge[0].getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-        || (!Bytes.equals(regionsToMerge[1].getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
-            && Bytes.compareTo(regionsToMerge[0].getEndKey(), regionsToMerge[1].getEndKey()) > 0)) {
-      endKey = regionsToMerge[0].getEndKey();
-    } else {
-      endKey = regionsToMerge[1].getEndKey();
-    }
-
-    // Merged region is sorted between two merging regions in META
-    mergedRegionInfo = new HRegionInfo(getTableName(), startKey, endKey, false, rid);
-  }
-
-  /**
-   * Move all regions to the same region server
-   * @param env MasterProcedureEnv
-   * @return whether target regions hosted by the same RS
-   * @throws IOException
-   */
-  private boolean MoveRegionsToSameRS(final MasterProcedureEnv env) throws IOException {
-    // Make sure regions are on the same regionserver before send merge
-    // regions request to region server.
-    //
-    boolean onSameRS = isRegionsOnTheSameServer(env);
-    if (!onSameRS) {
-      // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
-      // if we want to extend to more than 2 regions, the code needs to modify a little bit.
-      //
-      RegionStates regionStates = getAssignmentManager(env).getRegionStates();
-      ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
-
-      RegionLoad loadOfRegionA = getRegionLoad(env, regionLocation, regionsToMerge[0]);
-      RegionLoad loadOfRegionB = getRegionLoad(env, regionLocation2, regionsToMerge[1]);
-      if (loadOfRegionA != null && loadOfRegionB != null
-          && loadOfRegionA.getRequestsCount() < loadOfRegionB.getRequestsCount()) {
-        // switch regionsToMerge[0] and regionsToMerge[1]
-        HRegionInfo tmpRegion = this.regionsToMerge[0];
-        this.regionsToMerge[0] = this.regionsToMerge[1];
-        this.regionsToMerge[1] = tmpRegion;
-        ServerName tmpLocation = regionLocation;
-        regionLocation = regionLocation2;
-        regionLocation2 = tmpLocation;
-      }
-
-      long startTime = EnvironmentEdgeManager.currentTime();
-
-      RegionPlan regionPlan = new RegionPlan(regionsToMerge[1], regionLocation2, regionLocation);
-      LOG.info("Moving regions to same server for merge: " + regionPlan.toString());
-      getAssignmentManager(env).balance(regionPlan);
-      do {
-        try {
-          Thread.sleep(20);
-          // Make sure check RIT first, then get region location, otherwise
-          // we would make a wrong result if region is online between getting
-          // region location and checking RIT
-          boolean isRIT = regionStates.isRegionInTransition(regionsToMerge[1]);
-          regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[1]);
-          onSameRS = regionLocation.equals(regionLocation2);
-          if (onSameRS || !isRIT) {
-            // Regions are on the same RS, or regionsToMerge[1] is not in
-            // RegionInTransition any more
-            break;
-          }
-        } catch (InterruptedException e) {
-          InterruptedIOException iioe = new InterruptedIOException();
-          iioe.initCause(e);
-          throw iioe;
-        }
-      } while ((EnvironmentEdgeManager.currentTime() - startTime) <= getTimeout(env));
-    }
-    return onSameRS;
-  }
-
-  /**
-   * Pre merge region action
-   * @param env MasterProcedureEnv
-   **/
-  private void preMergeRegions(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
-      if (ret) {
-        throw new IOException(
-          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
-      }
-    }
-  }
-
-  /**
-   * Action after rollback a merge table regions action.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void postRollBackMergeRegions(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.postRollBackMergeRegionsAction(regionsToMerge, getUser());
-    }
-  }
-
-  /**
-   * Set the region states to MERGING state
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  public void setRegionStateToMerging(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.READY_TO_MERGE);
-    transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
-    if (env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getServerName(env), transition.build()) != null) {
-      throw new IOException("Failed to update region state to MERGING for "
-          + getRegionsToMergeListFullNameString());
-    }
-  }
-
-  /**
-   * Rollback the region state change
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void setRegionStateToRevertMerging(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.MERGE_REVERTED);
-    transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
-    String msg = env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getServerName(env), transition.build());
-    if (msg != null) {
-      // If daughter regions are online, the msg is coming from RPC retry.  Ignore it.
-      RegionStates regionStates = getAssignmentManager(env).getRegionStates();
-      if (!regionStates.isRegionOnline(regionsToMerge[0]) ||
-          !regionStates.isRegionOnline(regionsToMerge[1])) {
-        throw new IOException("Failed to update region state for "
-          + getRegionsToMergeListFullNameString()
-          + " as part of operation for reverting merge.  Error message: " + msg);
-      }
-    }
-  }
-
-  /**
-   * Create merged region
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
-    final FileSystem fs = mfs.getFileSystem();
-    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
-    regionFs.createMergesDir();
-
-    mergeStoreFiles(env, regionFs, regionFs.getMergesDir());
-    HRegionFileSystem regionFs2 = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[1], false);
-    mergeStoreFiles(env, regionFs2, regionFs.getMergesDir());
-
-    regionFs.commitMergedRegion(mergedRegionInfo);
-  }
-
-  /**
-   * Create reference file(s) of merging regions under the merges directory
-   * @param env MasterProcedureEnv
-   * @param regionFs region file system
-   * @param mergedDir the temp directory of merged region
-   * @throws IOException
-   */
-  private void mergeStoreFiles(
-      final MasterProcedureEnv env, final HRegionFileSystem regionFs, final Path mergedDir)
-      throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Configuration conf = env.getMasterConfiguration();
-    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
-
-    for (String family: regionFs.getFamilies()) {
-      final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
-      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
-
-      if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          // Create reference file(s) of the region in mergedDir
-          regionFs.mergeStoreFile(
-            mergedRegionInfo,
-            family,
-            new StoreFile(
-              mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
-            mergedDir);
-        }
-      }
-    }
-  }
-
-  /**
-   * Clean up merged region
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
-    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
-    final FileSystem fs = mfs.getFileSystem();
-    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
-    regionFs.cleanupMergedRegion(mergedRegionInfo);
-  }
-
-  /**
-   * RPC to region server that host the regions to merge, ask for close these regions
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void closeRegionsForMerge(final MasterProcedureEnv env) throws IOException {
-    boolean success = env.getMasterServices().getServerManager().sendRegionCloseForSplitOrMerge(
-      getServerName(env), regionsToMerge[0], regionsToMerge[1]);
-    if (!success) {
-      throw new IOException("Close regions " + getRegionsToMergeListFullNameString()
-          + " for merging failed. Check region server log for more details.");
-    }
-  }
-
-  /**
-   * Rollback close regions
-   * @param env MasterProcedureEnv
-   **/
-  private void rollbackCloseRegionsForMerge(final MasterProcedureEnv env) throws IOException {
-    // Check whether the region is closed; if so, open it in the same server
-    RegionStates regionStates = getAssignmentManager(env).getRegionStates();
-    for(int i = 1; i < regionsToMerge.length; i++) {
-      RegionState state = regionStates.getRegionState(regionsToMerge[i]);
-      if (state != null && (state.isClosing() || state.isClosed())) {
-        env.getMasterServices().getServerManager().sendRegionOpen(
-          getServerName(env),
-          regionsToMerge[i],
-          ServerName.EMPTY_SERVER_LIST);
-      }
-    }
-  }
-
-  /**
-   * Post merge region action
-   * @param env MasterProcedureEnv
-   **/
-  private void preMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      @MetaMutationAnnotation
-      final List<Mutation> metaEntries = new ArrayList<>();
-      boolean ret = cpHost.preMergeRegionsCommit(regionsToMerge, metaEntries, getUser());
-
-      if (ret) {
-        throw new IOException(
-          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
-      }
-      try {
-        for (Mutation p : metaEntries) {
-          HRegionInfo.parseRegionName(p.getRow());
-        }
-      } catch (IOException e) {
-        LOG.error("Row key of mutation from coprocessor is not parsable as region name."
-          + "Mutations from coprocessor should only be for hbase:meta table.", e);
-        throw e;
-      }
-    }
-  }
-
-  /**
-   * Add merged region to META and delete original regions.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException {
-    RegionStateTransition.Builder transition = RegionStateTransition.newBuilder();
-    transition.setTransitionCode(TransitionCode.MERGE_PONR);
-    transition.addRegionInfo(HRegionInfo.convert(mergedRegionInfo));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[0]));
-    transition.addRegionInfo(HRegionInfo.convert(regionsToMerge[1]));
-    // Add merged region and delete original regions
-    // as an atomic update. See HBASE-7721. This update to hbase:meta makes the region
-    // will determine whether the region is merged or not in case of failures.
-    if (env.getMasterServices().getAssignmentManager().onRegionTransition(
-      getServerName(env), transition.build()) != null) {
-      throw new IOException("Failed to update meta to add merged region that merges "
-          + getRegionsToMergeListFullNameString());
-    }
-  }
-
-  /**
-   * Post merge region action
-   * @param env MasterProcedureEnv
-   **/
-  private void postMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.postMergeRegionsCommit(regionsToMerge, mergedRegionInfo, getUser());
-    }
-  }
-
-  /**
-   * Assign merged region
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   * @throws InterruptedException
-   **/
-  private void openMergedRegions(final MasterProcedureEnv env)
-      throws IOException, InterruptedException {
-    // Check whether the merged region is already opened; if so,
-    // this is retry and we should just ignore.
-    RegionState regionState =
-        getAssignmentManager(env).getRegionStates().getRegionState(mergedRegionInfo);
-    if (regionState != null && regionState.isOpened()) {
-      LOG.info("Skip opening merged region " + mergedRegionInfo.getRegionNameAsString()
-        + " as it is already opened.");
-      return;
-    }
-
-    // TODO: The new AM should provide an API to force assign the merged region to the same RS
-    // as daughter regions; if the RS is unavailable, then assign to a different RS.
-    env.getMasterServices().getAssignmentManager().assignMergedRegion(
-      mergedRegionInfo, regionsToMerge[0], regionsToMerge[1]);
-  }
-
-  /**
-   * Post merge region action
-   * @param env MasterProcedureEnv
-   **/
-  private void postCompletedMergeRegions(final MasterProcedureEnv env) throws IOException {
-    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
-    if (cpHost != null) {
-      cpHost.postCompletedMergeRegionsAction(regionsToMerge, mergedRegionInfo, getUser());
-    }
-  }
-
-  private RegionLoad getRegionLoad(
-      final MasterProcedureEnv env,
-      final ServerName sn,
-      final HRegionInfo hri) {
-    ServerManager serverManager =  env.getMasterServices().getServerManager();
-    ServerLoad load = serverManager.getLoad(sn);
-    if (load != null) {
-      Map<byte[], RegionLoad> regionsLoad = load.getRegionsLoad();
-      if (regionsLoad != null) {
-        return regionsLoad.get(hri.getRegionName());
-      }
-    }
-    return null;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @param env MasterProcedureEnv
-   * @return whether target regions hosted by the same RS
-   */
-  private boolean isRegionsOnTheSameServer(final MasterProcedureEnv env) throws IOException{
-    Boolean onSameRS = true;
-    int i = 0;
-    RegionStates regionStates = getAssignmentManager(env).getRegionStates();
-    regionLocation = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
-    if (regionLocation != null) {
-      for(i = 1; i < regionsToMerge.length; i++) {
-        ServerName regionLocation2 = regionStates.getRegionServerOfRegion(regionsToMerge[i]);
-        if (regionLocation2 != null) {
-          if (onSameRS) {
-            onSameRS = regionLocation.equals(regionLocation2);
-          }
-        } else {
-          // At least one region is not online, merge will fail, no need to continue.
-          break;
-        }
-      }
-      if (i == regionsToMerge.length) {
-        // Finish checking all regions, return the result;
-        return onSameRS;
-      }
-    }
-
-    // If reaching here, at least one region is not online.
-    String msg = "Skip merging regions " + getRegionsToMergeListFullNameString() +
-        ", because region " + regionsToMerge[i].getEncodedName() + " is not online now.";
-    LOG.warn(msg);
-    throw new IOException(msg);
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @param env MasterProcedureEnv
-   * @return assignmentManager
-   */
-  private AssignmentManager getAssignmentManager(final MasterProcedureEnv env) {
-    if (assignmentManager == null) {
-      assignmentManager = env.getMasterServices().getAssignmentManager();
-    }
-    return assignmentManager;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @param env MasterProcedureEnv
-   * @return timeout value
-   */
-  private int getTimeout(final MasterProcedureEnv env) {
-    if (timeout == -1) {
-      timeout = env.getMasterConfiguration().getInt(
-        "hbase.master.regionmerge.timeout", regionsToMerge.length * 60 * 1000);
-    }
-    return timeout;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @param env MasterProcedureEnv
-   * @return serverName
-   */
-  private ServerName getServerName(final MasterProcedureEnv env) {
-    if (regionLocation == null) {
-      regionLocation =
-          getAssignmentManager(env).getRegionStates().getRegionServerOfRegion(regionsToMerge[0]);
-    }
-    return regionLocation;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @param fullName whether return only encoded name
-   * @return region names in a list
-   */
-  private String getRegionsToMergeListFullNameString() {
-    if (regionsToMergeListFullName == null) {
-      StringBuilder sb = new StringBuilder("[");
-      int i = 0;
-      while(i < regionsToMerge.length - 1) {
-        sb.append(regionsToMerge[i].getRegionNameAsString() + ", ");
-        i++;
-      }
-      sb.append(regionsToMerge[i].getRegionNameAsString() + " ]");
-      regionsToMergeListFullName = sb.toString();
-    }
-    return regionsToMergeListFullName;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return encoded region names
-   */
-  private String getRegionsToMergeListEncodedNameString() {
-    if (regionsToMergeListEncodedName == null) {
-      StringBuilder sb = new StringBuilder("[");
-      int i = 0;
-      while(i < regionsToMerge.length - 1) {
-        sb.append(regionsToMerge[i].getEncodedName() + ", ");
-        i++;
-      }
-      sb.append(regionsToMerge[i].getEncodedName() + " ]");
-      regionsToMergeListEncodedName = sb.toString();
-    }
-    return regionsToMergeListEncodedName;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return traceEnabled
-   */
-  private Boolean isTraceEnabled() {
-    if (traceEnabled == null) {
-      traceEnabled = LOG.isTraceEnabled();
-    }
-    return traceEnabled;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
index 52bb4d5..622c19f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
@@ -21,17 +21,14 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -97,7 +94,9 @@ public class ModifyColumnFamilyProcedure
         setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        reOpenAllRegionsIfTableIsOnline(env);
+        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
+          addChildProcedure(env.getAssignmentManager().createReopenProcedures(getTableName()));
+        }
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -265,7 +264,8 @@ public class ModifyColumnFamilyProcedure
     env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    reOpenAllRegionsIfTableIsOnline(env);
+    //reOpenAllRegionsIfTableIsOnline(env);
+    // TODO: NUKE ROLLBACK!!!!
   }
 
   /**
@@ -281,26 +281,6 @@ public class ModifyColumnFamilyProcedure
   }
 
   /**
-   * Last action from the procedure - executed when online schema change is supported.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
-    // This operation only run when the table is enabled.
-    if (!env.getMasterServices().getTableStateManager()
-        .isTableState(getTableName(), TableState.State.ENABLED)) {
-      return;
-    }
-
-    List<HRegionInfo> regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
-    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
-      LOG.info("Completed add column family operation on table " + getTableName());
-    } else {
-      LOG.warn("Error on reopening the regions on table " + getTableName());
-    }
-  }
-
-  /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
    * @return traceEnabled

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 6a70f62..20a6a03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -120,7 +120,10 @@ public class ModifyTableProcedure
         setNextState(ModifyTableState.MODIFY_TABLE_REOPEN_ALL_REGIONS);
         break;
       case MODIFY_TABLE_REOPEN_ALL_REGIONS:
-        reOpenAllRegionsIfTableIsOnline(env);
+        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
+          addChildProcedure(env.getAssignmentManager()
+            .createReopenProcedures(getRegionInfoList(env)));
+        }
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -299,7 +302,8 @@ public class ModifyTableProcedure
     deleteFromFs(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    reOpenAllRegionsIfTableIsOnline(env);
+    //reOpenAllRegionsIfTableIsOnline(env);
+    // TODO: NUKE ROLLBACK!!!!
   }
 
   /**
@@ -374,25 +378,6 @@ public class ModifyTableProcedure
   }
 
   /**
-   * Last action from the procedure - executed when online schema change is supported.
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
-    // This operation only run when the table is enabled.
-    if (!env.getMasterServices().getTableStateManager()
-        .isTableState(getTableName(), TableState.State.ENABLED)) {
-      return;
-    }
-
-    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), getRegionInfoList(env))) {
-      LOG.info("Completed modify table operation on table " + getTableName());
-    } else {
-      LOG.warn("Error on reopening the regions on table " + getTableName());
-    }
-  }
-
-  /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
    * @return traceEnabled whether the trace is enabled
@@ -430,7 +415,8 @@ public class ModifyTableProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+      regionInfoList = env.getAssignmentManager().getRegionStates()
+          .getRegionsOfTable(getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index 3777c79..1542f21 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -21,30 +21,26 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
-import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 
 /**
  * Helper to synchronously wait on conditions.
@@ -64,19 +60,93 @@ public final class ProcedureSyncWait {
     T evaluate() throws IOException;
   }
 
+  private static class ProcedureFuture implements Future<byte[]> {
+      private final ProcedureExecutor<MasterProcedureEnv> procExec;
+      private final long procId;
+
+      private boolean hasResult = false;
+      private byte[] result = null;
+
+      public ProcedureFuture(ProcedureExecutor<MasterProcedureEnv> procExec, long procId) {
+        this.procExec = procExec;
+        this.procId = procId;
+      }
+
+      @Override
+      public boolean cancel(boolean mayInterruptIfRunning) { return false; }
+
+      @Override
+      public boolean isCancelled() { return false; }
+
+      @Override
+      public boolean isDone() { return hasResult; }
+
+      @Override
+      public byte[] get() throws InterruptedException, ExecutionException {
+        if (hasResult) return result;
+        try {
+          return waitForProcedureToComplete(procExec, procId, Long.MAX_VALUE);
+        } catch (Exception e) {
+          throw new ExecutionException(e);
+        }
+      }
+
+      @Override
+      public byte[] get(long timeout, TimeUnit unit)
+          throws InterruptedException, ExecutionException, TimeoutException {
+        if (hasResult) return result;
+        try {
+          result = waitForProcedureToComplete(procExec, procId, unit.toMillis(timeout));
+          hasResult = true;
+          return result;
+        } catch (TimeoutIOException e) {
+          throw new TimeoutException(e.getMessage());
+        } catch (Exception e) {
+          throw new ExecutionException(e);
+        }
+      }
+    }
+
+  public static Future<byte[]> submitProcedure(final ProcedureExecutor<MasterProcedureEnv> procExec,
+      final Procedure proc) {
+    if (proc.isInitializing()) {
+      procExec.submitProcedure(proc);
+    }
+    return new ProcedureFuture(procExec, proc.getProcId());
+  }
+
   public static byte[] submitAndWaitProcedure(ProcedureExecutor<MasterProcedureEnv> procExec,
       final Procedure proc) throws IOException {
-    long procId = procExec.submitProcedure(proc);
-    return waitForProcedureToComplete(procExec, procId);
+    if (proc.isInitializing()) {
+      procExec.submitProcedure(proc);
+    }
+    return waitForProcedureToCompleteIOE(procExec, proc.getProcId(), Long.MAX_VALUE);
   }
 
-  private static byte[] waitForProcedureToComplete(ProcedureExecutor<MasterProcedureEnv> procExec,
-      final long procId) throws IOException {
-    while (!procExec.isFinished(procId) && procExec.isRunning()) {
-      // TODO: add a config to make it tunable
-      // Dev Consideration: are we waiting forever, or we can set up some timeout value?
-      Threads.sleepWithoutInterrupt(250);
+  public static byte[] waitForProcedureToCompleteIOE(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId, final long timeout)
+  throws IOException {
+    try {
+      return waitForProcedureToComplete(procExec, procId, timeout);
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IOException(e);
     }
+  }
+
+  public static byte[] waitForProcedureToComplete(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId, final long timeout)
+      throws IOException {
+    waitFor(procExec.getEnvironment(), "procId=" + procId,
+      new ProcedureSyncWait.Predicate<Boolean>() {
+        @Override
+        public Boolean evaluate() throws IOException {
+          return !procExec.isRunning() || procExec.isFinished(procId);
+        }
+      }
+    );
+
     ProcedureInfo result = procExec.getResult(procId);
     if (result != null) {
       if (result.isFailed()) {
@@ -104,6 +174,7 @@ public final class ProcedureSyncWait {
   public static <T> T waitFor(MasterProcedureEnv env, long waitTime, long waitingTimeForEvents,
       String purpose, Predicate<T> predicate) throws IOException {
     final long done = EnvironmentEdgeManager.currentTime() + waitTime;
+    boolean logged = false;
     do {
       T result = predicate.evaluate();
       if (result != null && !result.equals(Boolean.FALSE)) {
@@ -115,7 +186,12 @@ public final class ProcedureSyncWait {
         LOG.warn("Interrupted while sleeping, waiting on " + purpose);
         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
       }
-      LOG.debug("Waiting on " + purpose);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Waiting on " + purpose);
+      } else {
+        if (!logged) LOG.debug("Waiting on " + purpose);
+      }
+      logged = true;
     } while (EnvironmentEdgeManager.currentTime() < done && env.isRunning());
 
     throw new TimeoutIOException("Timed out while waiting on " + purpose);
@@ -133,44 +209,14 @@ public final class ProcedureSyncWait {
     }
   }
 
-  protected static void waitRegionServers(final MasterProcedureEnv env) throws IOException {
-    final ServerManager sm = env.getMasterServices().getServerManager();
-    ProcedureSyncWait.waitFor(env, "server to assign region(s)",
-        new ProcedureSyncWait.Predicate<Boolean>() {
-      @Override
-      public Boolean evaluate() throws IOException {
-        List<ServerName> servers = sm.createDestinationServersList();
-        return servers != null && !servers.isEmpty();
-      }
-    });
-  }
-
-  protected static List<HRegionInfo> getRegionsFromMeta(final MasterProcedureEnv env,
-      final TableName tableName) throws IOException {
-    return ProcedureSyncWait.waitFor(env, "regions of table=" + tableName + " from meta",
-        new ProcedureSyncWait.Predicate<List<HRegionInfo>>() {
-      @Override
-      public List<HRegionInfo> evaluate() throws IOException {
-        if (TableName.META_TABLE_NAME.equals(tableName)) {
-          return new MetaTableLocator().getMetaRegions(env.getMasterServices().getZooKeeper());
-        }
-        return MetaTableAccessor.getTableRegions(env.getMasterServices().getConnection(),tableName);
-      }
-    });
-  }
-
   protected static void waitRegionInTransition(final MasterProcedureEnv env,
       final List<HRegionInfo> regions) throws IOException, CoordinatedStateException {
-    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    final RegionStates states = am.getRegionStates();
+    final RegionStates states = env.getAssignmentManager().getRegionStates();
     for (final HRegionInfo region : regions) {
       ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
           new ProcedureSyncWait.Predicate<Boolean>() {
         @Override
         public Boolean evaluate() throws IOException {
-          if (states.isRegionInState(region, State.FAILED_OPEN)) {
-            am.regionOffline(region);
-          }
           return !states.isRegionInTransition(region);
         }
       });

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
new file mode 100644
index 0000000..be52aea
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -0,0 +1,542 @@
+/**
+ * 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 com.google.common.collect.ArrayListMultimap;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+/**
+ * A remote procecdure dispatcher for regionservers.
+ */
+public class RSProcedureDispatcher
+    extends RemoteProcedureDispatcher<MasterProcedureEnv, ServerName>
+    implements ServerListener {
+  private static final Log LOG = LogFactory.getLog(RSProcedureDispatcher.class);
+
+  public static final String RS_RPC_STARTUP_WAIT_TIME_CONF_KEY =
+      "hbase.regionserver.rpc.startup.waittime";
+  private static final int DEFAULT_RS_RPC_STARTUP_WAIT_TIME = 60000;
+
+  private static final int RS_VERSION_WITH_EXEC_PROCS = 0x0201000; // 2.1
+
+  protected final MasterServices master;
+  protected final long rsStartupWaitTime;
+
+  public RSProcedureDispatcher(final MasterServices master) {
+    super(master.getConfiguration());
+
+    this.master = master;
+    this.rsStartupWaitTime = master.getConfiguration().getLong(
+      RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, DEFAULT_RS_RPC_STARTUP_WAIT_TIME);
+  }
+
+  @Override
+  public boolean start() {
+    if (!super.start()) {
+      return false;
+    }
+
+    master.getServerManager().registerListener(this);
+    for (ServerName serverName: master.getServerManager().getOnlineServersList()) {
+      addNode(serverName);
+    }
+    return true;
+  }
+
+  @Override
+  public boolean stop() {
+    if (!super.stop()) {
+      return false;
+    }
+
+    master.getServerManager().unregisterListener(this);
+    return true;
+  }
+
+  @Override
+  protected void remoteDispatch(final ServerName serverName,
+      final Set<RemoteProcedure> operations) {
+    final int rsVersion = master.getAssignmentManager().getServerVersion(serverName);
+    /*
+    if (rsVersion >= RS_VERSION_WITH_EXEC_PROCS) {
+      LOG.info(String.format(
+        "use the procedure batch rpc execution for serverName=%s version=%s",
+        serverName, rsVersion));
+      submitTask(new ExecuteProceduresRemoteCall(serverName, operations));
+    }
+    */
+    LOG.info(String.format(
+      "Fallback to compat rpc execution for serverName=%s version=%s",
+      serverName, rsVersion));
+    submitTask(new CompatRemoteProcedureResolver(serverName, operations));
+  }
+
+  protected void abortPendingOperations(final ServerName serverName,
+      final Set<RemoteProcedure> operations) {
+    // TODO: Replace with a ServerNotOnlineException()
+    final IOException e = new DoNotRetryIOException("server not online " + serverName);
+    final MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    for (RemoteProcedure proc: operations) {
+      proc.remoteCallFailed(env, serverName, e);
+    }
+  }
+
+  public void serverAdded(final ServerName serverName) {
+    addNode(serverName);
+  }
+
+  public void serverRemoved(final ServerName serverName) {
+    removeNode(serverName);
+  }
+
+  /**
+   * Base remote call
+   */
+  protected abstract class AbstractRSRemoteCall implements Callable<Void> {
+    private final ServerName serverName;
+
+    private int numberOfAttemptsSoFar = 0;
+    private long maxWaitTime = -1;
+
+    public AbstractRSRemoteCall(final ServerName serverName) {
+      this.serverName = serverName;
+    }
+
+    public abstract Void call();
+
+    protected AdminService.BlockingInterface getRsAdmin() throws IOException {
+      final AdminService.BlockingInterface admin = master.getServerManager().getRsAdmin(serverName);
+      if (admin == null) {
+        throw new IOException("Attempting to send OPEN RPC to server " + getServerName() +
+          " failed because no RPC connection found to this server");
+      }
+      return admin;
+    }
+
+    protected ServerName getServerName() {
+      return serverName;
+    }
+
+    protected boolean scheduleForRetry(final IOException e) {
+      // Should we wait a little before retrying? If the server is starting it's yes.
+      final boolean hold = (e instanceof ServerNotRunningYetException);
+      if (hold) {
+        LOG.warn(String.format("waiting a little before trying on the same server=%s try=%d",
+            serverName, numberOfAttemptsSoFar), e);
+        long now = EnvironmentEdgeManager.currentTime();
+        if (now < getMaxWaitTime()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("server is not yet up; waiting up to %dms",
+              (getMaxWaitTime() - now)), e);
+          }
+          submitTask(this, 100, TimeUnit.MILLISECONDS);
+          return true;
+        }
+
+        LOG.warn(String.format("server %s is not up for a while; try a new one", serverName), e);
+        return false;
+      }
+
+      // In case socket is timed out and the region server is still online,
+      // the openRegion RPC could have been accepted by the server and
+      // just the response didn't go through.  So we will retry to
+      // open the region on the same server.
+      final boolean retry = !hold && (e instanceof SocketTimeoutException
+          && master.getServerManager().isServerOnline(serverName));
+      if (retry) {
+        // we want to retry as many times as needed as long as the RS is not dead.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("Retrying to same RegionServer %s because: %s",
+              serverName, e.getMessage()), e);
+        }
+        submitTask(this);
+        return true;
+      }
+
+      // trying to send the request elsewhere instead
+      LOG.warn(String.format("the request should be tried elsewhere instead; server=%s try=%d",
+                  serverName, numberOfAttemptsSoFar), e);
+      return false;
+    }
+
+    private long getMaxWaitTime() {
+      if (this.maxWaitTime < 0) {
+        // This is the max attempts, not retries, so it should be at least 1.
+        this.maxWaitTime = EnvironmentEdgeManager.currentTime() + rsStartupWaitTime;
+      }
+      return this.maxWaitTime;
+    }
+
+    protected IOException unwrapException(IOException e) {
+      if (e instanceof RemoteException) {
+        e = ((RemoteException)e).unwrapRemoteException();
+      }
+      return e;
+    }
+  }
+
+  private interface RemoteProcedureResolver {
+    void dispatchOpenRequests(MasterProcedureEnv env, List<RegionOpenOperation> operations);
+    void dispatchCloseRequests(MasterProcedureEnv env, List<RegionCloseOperation> operations);
+  }
+
+  public void splitAndResolveOperation(final ServerName serverName,
+      final Set<RemoteProcedure> operations, final RemoteProcedureResolver resolver) {
+    final MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    final ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
+      buildAndGroupRequestByType(env, serverName, operations);
+
+    final List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
+    if (!openOps.isEmpty()) resolver.dispatchOpenRequests(env, openOps);
+
+    final List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
+    if (!closeOps.isEmpty()) resolver.dispatchCloseRequests(env, closeOps);
+
+    if (!reqsByType.isEmpty()) {
+      LOG.warn("unknown request type in the queue: " + reqsByType);
+    }
+  }
+
+  // ==========================================================================
+  //  Compatibility calls
+  // ==========================================================================
+  protected class ExecuteProceduresRemoteCall extends AbstractRSRemoteCall
+      implements RemoteProcedureResolver {
+    private final Set<RemoteProcedure> operations;
+
+    private ExecuteProceduresRequest.Builder request = null;
+
+    public ExecuteProceduresRemoteCall(final ServerName serverName,
+        final Set<RemoteProcedure> operations) {
+      super(serverName);
+      this.operations = operations;
+    }
+
+    public Void call() {
+      final MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+
+      request = ExecuteProceduresRequest.newBuilder();
+      splitAndResolveOperation(getServerName(), operations, this);
+
+      try {
+        final ExecuteProceduresResponse response = sendRequest(getServerName(), request.build());
+        remoteCallCompleted(env, response);
+      } catch (IOException e) {
+        e = unwrapException(e);
+        // TODO: In the future some operation may want to bail out early.
+        // TODO: How many times should we retry (use numberOfAttemptsSoFar)
+        if (!scheduleForRetry(e)) {
+          remoteCallFailed(env, e);
+        }
+      }
+      return null;
+    }
+
+    public void dispatchOpenRequests(final MasterProcedureEnv env,
+        final List<RegionOpenOperation> operations) {
+      request.addOpenRegion(buildOpenRegionRequest(env, getServerName(), operations));
+    }
+
+    public void dispatchCloseRequests(final MasterProcedureEnv env,
+        final List<RegionCloseOperation> operations) {
+      for (RegionCloseOperation op: operations) {
+        request.addCloseRegion(op.buildCloseRegionRequest(getServerName()));
+      }
+    }
+
+    protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
+        final ExecuteProceduresRequest request) throws IOException {
+      try {
+        return getRsAdmin().executeProcedures(null, request);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+    }
+
+
+    private void remoteCallCompleted(final MasterProcedureEnv env,
+        final ExecuteProceduresResponse response) {
+      /*
+      for (RemoteProcedure proc: operations) {
+        proc.remoteCallCompleted(env, getServerName(), response);
+      }*/
+    }
+
+    private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
+      for (RemoteProcedure proc: operations) {
+        proc.remoteCallFailed(env, getServerName(), e);
+      }
+    }
+  }
+
+  // ==========================================================================
+  //  Compatibility calls
+  //  Since we don't have a "batch proc-exec" request on the target RS
+  //  we have to chunk the requests by type and dispatch the specific request.
+  // ==========================================================================
+  private static OpenRegionRequest buildOpenRegionRequest(final MasterProcedureEnv env,
+      final ServerName serverName, final List<RegionOpenOperation> operations) {
+    final OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
+    builder.setServerStartCode(serverName.getStartcode());
+    builder.setMasterSystemTime(EnvironmentEdgeManager.currentTime());
+    for (RegionOpenOperation op: operations) {
+      builder.addOpenInfo(op.buildRegionOpenInfoRequest(env));
+    }
+    return builder.build();
+  }
+
+  private final class OpenRegionRemoteCall extends AbstractRSRemoteCall {
+    private final List<RegionOpenOperation> operations;
+
+    public OpenRegionRemoteCall(final ServerName serverName,
+        final List<RegionOpenOperation> operations) {
+      super(serverName);
+      this.operations = operations;
+    }
+
+    @Override
+    public Void call() {
+      final MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+      final OpenRegionRequest request = buildOpenRegionRequest(env, getServerName(), operations);
+
+      try {
+        OpenRegionResponse response = sendRequest(getServerName(), request);
+        remoteCallCompleted(env, response);
+      } catch (IOException e) {
+        e = unwrapException(e);
+        // TODO: In the future some operation may want to bail out early.
+        // TODO: How many times should we retry (use numberOfAttemptsSoFar)
+        if (!scheduleForRetry(e)) {
+          remoteCallFailed(env, e);
+        }
+      }
+      return null;
+    }
+
+    private OpenRegionResponse sendRequest(final ServerName serverName,
+        final OpenRegionRequest request) throws IOException {
+      try {
+        return getRsAdmin().openRegion(null, request);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+    }
+
+    private void remoteCallCompleted(final MasterProcedureEnv env,
+        final OpenRegionResponse response) {
+      int index = 0;
+      for (RegionOpenOperation op: operations) {
+        OpenRegionResponse.RegionOpeningState state = response.getOpeningState(index++);
+        op.setFailedOpen(state == OpenRegionResponse.RegionOpeningState.FAILED_OPENING);
+        op.getRemoteProcedure().remoteCallCompleted(env, getServerName(), op);
+      }
+    }
+
+    private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
+      for (RegionOpenOperation op: operations) {
+        op.getRemoteProcedure().remoteCallFailed(env, getServerName(), e);
+      }
+    }
+  }
+
+  private final class CloseRegionRemoteCall extends AbstractRSRemoteCall {
+    private final RegionCloseOperation operation;
+
+    public CloseRegionRemoteCall(final ServerName serverName,
+        final RegionCloseOperation operation) {
+      super(serverName);
+      this.operation = operation;
+    }
+
+    @Override
+    public Void call() {
+      final MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+      final CloseRegionRequest request = operation.buildCloseRegionRequest(getServerName());
+      try {
+        CloseRegionResponse response = sendRequest(getServerName(), request);
+        remoteCallCompleted(env, response);
+      } catch (IOException e) {
+        e = unwrapException(e);
+        // TODO: In the future some operation may want to bail out early.
+        // TODO: How many times should we retry (use numberOfAttemptsSoFar)
+        if (!scheduleForRetry(e)) {
+          remoteCallFailed(env, e);
+        }
+      }
+      return null;
+    }
+
+    private CloseRegionResponse sendRequest(final ServerName serverName,
+        final CloseRegionRequest request) throws IOException {
+      try {
+        return getRsAdmin().closeRegion(null, request);
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+    }
+
+    private void remoteCallCompleted(final MasterProcedureEnv env,
+        final CloseRegionResponse response) {
+      operation.setClosed(response.getClosed());
+      operation.getRemoteProcedure().remoteCallCompleted(env, getServerName(), operation);
+    }
+
+    private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
+      operation.getRemoteProcedure().remoteCallFailed(env, getServerName(), e);
+    }
+  }
+
+  protected class CompatRemoteProcedureResolver implements Callable<Void>, RemoteProcedureResolver {
+    private final Set<RemoteProcedure> operations;
+    private final ServerName serverName;
+
+    public CompatRemoteProcedureResolver(final ServerName serverName,
+        final Set<RemoteProcedure> operations) {
+      this.serverName = serverName;
+      this.operations = operations;
+    }
+
+    @Override
+    public Void call() {
+      splitAndResolveOperation(serverName, operations, this);
+      return null;
+    }
+
+    public void dispatchOpenRequests(final MasterProcedureEnv env,
+        final List<RegionOpenOperation> operations) {
+      submitTask(new OpenRegionRemoteCall(serverName, operations));
+    }
+
+    public void dispatchCloseRequests(final MasterProcedureEnv env,
+        final List<RegionCloseOperation> operations) {
+      for (RegionCloseOperation op: operations) {
+        submitTask(new CloseRegionRemoteCall(serverName, op));
+      }
+    }
+  }
+
+  // ==========================================================================
+  //  RPC Messages
+  //  - ServerOperation: refreshConfig, grant, revoke, ...
+  //  - RegionOperation: open, close, flush, snapshot, ...
+  // ==========================================================================
+  public static abstract class ServerOperation extends RemoteOperation {
+    protected ServerOperation(final RemoteProcedure remoteProcedure) {
+      super(remoteProcedure);
+    }
+  }
+
+  public static abstract class RegionOperation extends RemoteOperation {
+    private final HRegionInfo regionInfo;
+
+    protected RegionOperation(final RemoteProcedure remoteProcedure,
+        final HRegionInfo regionInfo) {
+      super(remoteProcedure);
+      this.regionInfo = regionInfo;
+    }
+
+    public HRegionInfo getRegionInfo() {
+      return this.regionInfo;
+    }
+  }
+
+  public static class RegionOpenOperation extends RegionOperation {
+    private final List<ServerName> favoredNodes;
+    private final boolean openForReplay;
+    private boolean failedOpen;
+
+    public RegionOpenOperation(final RemoteProcedure remoteProcedure,
+        final HRegionInfo regionInfo, final List<ServerName> favoredNodes,
+        final boolean openForReplay) {
+      super(remoteProcedure, regionInfo);
+      this.favoredNodes = favoredNodes;
+      this.openForReplay = openForReplay;
+    }
+
+    protected void setFailedOpen(final boolean failedOpen) {
+      this.failedOpen = failedOpen;
+    }
+
+    public boolean isFailedOpen() {
+      return failedOpen;
+    }
+
+    public OpenRegionRequest.RegionOpenInfo buildRegionOpenInfoRequest(
+        final MasterProcedureEnv env) {
+      return RequestConverter.buildRegionOpenInfo(getRegionInfo(),
+        env.getAssignmentManager().getFavoredNodes(getRegionInfo()), false);
+    }
+  }
+
+  public static class RegionCloseOperation extends RegionOperation {
+    private final ServerName destinationServer;
+    private boolean closed = false;
+
+    public RegionCloseOperation(final RemoteProcedure remoteProcedure,
+        final HRegionInfo regionInfo, final ServerName destinationServer) {
+      super(remoteProcedure, regionInfo);
+      this.destinationServer = destinationServer;
+    }
+
+    public ServerName getDestinationServer() {
+      return destinationServer;
+    }
+
+    protected void setClosed(final boolean closed) {
+      this.closed = closed;
+    }
+
+    public boolean isClosed() {
+      return closed;
+    }
+
+    public CloseRegionRequest buildCloseRegionRequest(final ServerName serverName) {
+      return ProtobufUtil.buildCloseRegionRequest(serverName,
+        getRegionInfo().getRegionName(), getDestinationServer());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index f8c9d8f..2281eb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MetricsSnapshot;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -405,17 +404,7 @@ public class RestoreSnapshotProcedure
     try {
       Connection conn = env.getMasterServices().getConnection();
 
-      // 1. Forces all the RegionStates to be offline
-      //
-      // The AssignmentManager keeps all the region states around
-      // with no possibility to remove them, until the master is restarted.
-      // This means that a region marked as SPLIT before the restore will never be assigned again.
-      // To avoid having all states around all the regions are switched to the OFFLINE state,
-      // which is the same state that the regions will be after a delete table.
-      forceRegionsOffline(env, regionsToAdd);
-      forceRegionsOffline(env, regionsToRestore);
-      forceRegionsOffline(env, regionsToRemove);
-
+      // 1. Prepare to restore
       getMonitorStatus().setStatus("Preparing to restore each region");
 
       // 2. Applies changes to hbase:meta
@@ -475,20 +464,6 @@ public class RestoreSnapshotProcedure
   }
 
   /**
-   * Make sure that region states of the region list is in OFFLINE state.
-   * @param env MasterProcedureEnv
-   * @param hris region info list
-   **/
-  private void forceRegionsOffline(final MasterProcedureEnv env, final List<HRegionInfo> hris) {
-    RegionStates states = env.getMasterServices().getAssignmentManager().getRegionStates();
-    if (hris != null) {
-      for (HRegionInfo hri: hris) {
-        states.regionOffline(hri);
-      }
-    }
-  }
-
-  /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
    * @return traceEnabled


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
deleted file mode 100644
index 929cd4e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
+++ /dev/null
@@ -1,122 +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;
-
-import java.io.IOException;
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.concurrent.Executors;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.Server;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Base class used bulk assigning and unassigning regions.
- * Encapsulates a fixed size thread pool of executors to run assignment/unassignment.
- * Implement {@link #populatePool(java.util.concurrent.ExecutorService)} and
- * {@link #waitUntilDone(long)}.  The default implementation of
- * the {@link #getUncaughtExceptionHandler()} is to abort the hosting
- * Server.
- */
-@InterfaceAudience.Private
-public abstract class BulkAssigner {
-  protected final Server server;
-
-  /**
-   * @param server An instance of Server
-   */
-  public BulkAssigner(final Server server) {
-    this.server = server;
-  }
-
-  /**
-   * @return What to use for a thread prefix when executor runs.
-   */
-  protected String getThreadNamePrefix() {
-    return this.server.getServerName() + "-" + this.getClass().getName(); 
-  }
-
-  protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
-    return new UncaughtExceptionHandler() {
-      @Override
-      public void uncaughtException(Thread t, Throwable e) {
-        // Abort if exception of any kind.
-        server.abort("Uncaught exception in " + t.getName(), e);
-      }
-    };
-  }
-
-  protected int getThreadCount() {
-    return this.server.getConfiguration().
-      getInt("hbase.bulk.assignment.threadpool.size", 20);
-  }
-
-  protected long getTimeoutOnRIT() {
-    return this.server.getConfiguration().
-      getLong("hbase.bulk.assignment.waiton.empty.rit", 5 * 60 * 1000);
-  }
-
-  protected abstract void populatePool(
-      final java.util.concurrent.ExecutorService pool) throws IOException;
-
-  public boolean bulkAssign() throws InterruptedException, IOException {
-    return bulkAssign(true);
-  }
-
-  /**
-   * Run the bulk assign.
-   * 
-   * @param sync
-   *          Whether to assign synchronously.
-   * @throws InterruptedException
-   * @return True if done.
-   * @throws IOException
-   */
-  public boolean bulkAssign(boolean sync) throws InterruptedException,
-      IOException {
-    boolean result = false;
-    ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
-    builder.setDaemon(true);
-    builder.setNameFormat(getThreadNamePrefix() + "-%1$d");
-    builder.setUncaughtExceptionHandler(getUncaughtExceptionHandler());
-    int threadCount = getThreadCount();
-    java.util.concurrent.ExecutorService pool =
-      Executors.newFixedThreadPool(threadCount, builder.build());
-    try {
-      populatePool(pool);
-      // How long to wait on empty regions-in-transition.  If we timeout, the
-      // RIT monitor should do fixup.
-      if (sync) result = waitUntilDone(getTimeoutOnRIT());
-    } finally {
-      // We're done with the pool.  It'll exit when its done all in queue.
-      pool.shutdown();
-    }
-    return result;
-  }
-
-  /**
-   * Wait until bulk assign is done.
-   * @param timeout How long to wait.
-   * @throws InterruptedException
-   * @return True if the condition we were waiting on happened.
-   */
-  protected abstract boolean waitUntilDone(final long timeout)
-  throws InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
deleted file mode 100644
index d8c511e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
+++ /dev/null
@@ -1,136 +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;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-
-/**
- * Performs bulk reopen of the list of regions provided to it.
- */
-@InterfaceAudience.Private
-public class BulkReOpen extends BulkAssigner {
-  private final Map<ServerName, List<HRegionInfo>> rsToRegions;
-  private final AssignmentManager assignmentManager;
-  private static final Log LOG = LogFactory.getLog(BulkReOpen.class);
-
-  public BulkReOpen(final Server server,
-      final Map<ServerName, List<HRegionInfo>> serverToRegions,
-    final AssignmentManager am) {
-    super(server);
-    this.assignmentManager = am;
-    this.rsToRegions = serverToRegions;
-  }
-
-  /**
-   * Unassign all regions, so that they go through the regular region
-   * assignment flow (in assignment manager) and are re-opened.
-   */
-  @Override
-  protected void populatePool(ExecutorService pool) {
-    LOG.debug("Creating threads for each region server ");
-    for (Map.Entry<ServerName, List<HRegionInfo>> e : rsToRegions
-        .entrySet()) {
-      final List<HRegionInfo> hris = e.getValue();
-      // add plans for the regions that need to be reopened
-      Map<String, RegionPlan> plans = new HashMap<>();
-      for (HRegionInfo hri : hris) {
-        RegionPlan reOpenPlan = assignmentManager.getRegionReopenPlan(hri);
-        plans.put(hri.getEncodedName(), reOpenPlan);
-      }
-      assignmentManager.addPlans(plans);
-      pool.execute(new Runnable() {
-        public void run() {
-          try {
-            unassign(hris);
-          } catch (Throwable t) {
-            LOG.warn("Failed bulking re-open " + hris.size()
-              + " region(s)", t);
-          }
-        }
-      });
-    }
-  }
-
- /**
-  * Reopen the regions asynchronously, so always returns true immediately.
-  * @return true
-  */
-  @Override
-  protected boolean waitUntilDone(long timeout) {
-    return true;
-  }
-
-  /**
-   * Configuration knobs "hbase.bulk.reopen.threadpool.size" number of regions
-   * that can be reopened concurrently. The maximum number of threads the master
-   * creates is never more than the number of region servers.
-   * If configuration is not defined it defaults to 20
-   */
-  protected int getThreadCount() {
-    int defaultThreadCount = super.getThreadCount();
-    return this.server.getConfiguration().getInt(
-        "hbase.bulk.reopen.threadpool.size", defaultThreadCount);
-  }
-
-  public boolean bulkReOpen() throws InterruptedException, IOException {
-    return bulkAssign();
-  }
-
-  /**
-   * Unassign the list of regions. Configuration knobs:
-   * hbase.bulk.waitbetween.reopen indicates the number of milliseconds to
-   * wait before unassigning another region from this region server
-   *
-   * @param regions
-   * @throws InterruptedException
-   */
-  private void unassign(
-      List<HRegionInfo> regions) throws InterruptedException {
-    int waitTime = this.server.getConfiguration().getInt(
-        "hbase.bulk.waitbetween.reopen", 0);
-    RegionStates regionStates = assignmentManager.getRegionStates();
-    for (HRegionInfo region : regions) {
-      if (server.isStopped()) {
-        return;
-      }
-      if (regionStates.isRegionInTransition(region)) {
-        continue;
-      }
-      assignmentManager.unassign(region);
-      while (regionStates.isRegionInTransition(region)
-          && !server.isStopped()) {
-        regionStates.waitForUpdate(100);
-      }
-      if (waitTime > 0 && !server.isStopped()) {
-        Thread.sleep(waitTime);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index affd44c..d682b82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -112,7 +113,7 @@ public class CatalogJanitor extends ScheduledChore {
           && !this.services.isInMaintenanceMode()
           && am != null
           && am.isFailoverCleanupDone()
-          && am.getRegionStates().getRegionsInTransition().isEmpty()) {
+          && am.hasRegionsInTransition()) {
         scan();
       } else {
         LOG.warn("CatalogJanitor disabled! Not running scan.");
@@ -167,6 +168,7 @@ public class CatalogJanitor extends ScheduledChore {
           // Another table, stop scanning
           return false;
         }
+        LOG.info("CATALOG INFO " + info + " IS-SPLIT_PARENT=" + info.isSplitParent());
         if (info.isSplitParent()) splitParents.put(info, r);
         if (r.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
           mergedRegions.put(info, r);
@@ -469,4 +471,4 @@ public class CatalogJanitor extends ScheduledChore {
     return cleanMergeRegion(region, mergeRegions.getFirst(),
         mergeRegions.getSecond());
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java
deleted file mode 100644
index fc3607f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java
+++ /dev/null
@@ -1,213 +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;
-
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-
-/**
- * Run bulk assign.  Does one RCP per regionserver passing a
- * batch of regions using {@link GeneralBulkAssigner.SingleServerBulkAssigner}.
- */
-@InterfaceAudience.Private
-public class GeneralBulkAssigner extends BulkAssigner {
-  private static final Log LOG = LogFactory.getLog(GeneralBulkAssigner.class);
-
-  private Map<ServerName, List<HRegionInfo>> failedPlans = new ConcurrentHashMap<>();
-  private ExecutorService pool;
-
-  final Map<ServerName, List<HRegionInfo>> bulkPlan;
-  final AssignmentManager assignmentManager;
-  final boolean waitTillAllAssigned;
-
-  public GeneralBulkAssigner(final Server server,
-      final Map<ServerName, List<HRegionInfo>> bulkPlan,
-      final AssignmentManager am, final boolean waitTillAllAssigned) {
-    super(server);
-    this.bulkPlan = bulkPlan;
-    this.assignmentManager = am;
-    this.waitTillAllAssigned = waitTillAllAssigned;
-  }
-
-  @Override
-  protected String getThreadNamePrefix() {
-    return this.server.getServerName() + "-GeneralBulkAssigner";
-  }
-
-  @Override
-  protected void populatePool(ExecutorService pool) {
-    this.pool = pool; // shut it down later in case some assigner hangs
-    for (Map.Entry<ServerName, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
-      pool.execute(new SingleServerBulkAssigner(e.getKey(), e.getValue(),
-        this.assignmentManager, this.failedPlans));
-    }
-  }
-
-  /**
-   *
-   * @param timeout How long to wait.
-   * @return true if done.
-   */
-  @Override
-  protected boolean waitUntilDone(final long timeout)
-  throws InterruptedException {
-    Set<HRegionInfo> regionSet = new HashSet<>();
-    for (List<HRegionInfo> regionList : bulkPlan.values()) {
-      regionSet.addAll(regionList);
-    }
-
-    pool.shutdown(); // no more task allowed
-    int serverCount = bulkPlan.size();
-    int regionCount = regionSet.size();
-    long startTime = System.currentTimeMillis();
-    long rpcWaitTime = startTime + timeout;
-    while (!server.isStopped() && !pool.isTerminated()
-        && rpcWaitTime > System.currentTimeMillis()) {
-      if (failedPlans.isEmpty()) {
-        pool.awaitTermination(100, TimeUnit.MILLISECONDS);
-      } else {
-        reassignFailedPlans();
-      }
-    }
-    if (!pool.isTerminated()) {
-      LOG.warn("bulk assigner is still running after "
-        + (System.currentTimeMillis() - startTime) + "ms, shut it down now");
-      // some assigner hangs, can't wait any more, shutdown the pool now
-      List<Runnable> notStarted = pool.shutdownNow();
-      if (notStarted != null && !notStarted.isEmpty()) {
-        server.abort("some single server assigner hasn't started yet"
-          + " when the bulk assigner timed out", null);
-        return false;
-      }
-    }
-
-    int reassigningRegions = 0;
-    if (!failedPlans.isEmpty() && !server.isStopped()) {
-      reassigningRegions = reassignFailedPlans();
-    }
-    assignmentManager.waitForAssignment(regionSet, waitTillAllAssigned,
-      reassigningRegions, Math.max(System.currentTimeMillis(), rpcWaitTime));
-
-    if (LOG.isDebugEnabled()) {
-      long elapsedTime = System.currentTimeMillis() - startTime;
-      String status = "successfully";
-      if (!regionSet.isEmpty()) {
-        status = "with " + regionSet.size() + " regions still in transition";
-      }
-      LOG.debug("bulk assigning total " + regionCount + " regions to "
-        + serverCount + " servers, took " + elapsedTime + "ms, " + status);
-    }
-    return regionSet.isEmpty();
-  }
-
-  @Override
-  protected long getTimeoutOnRIT() {
-    // Guess timeout.  Multiply the max number of regions on a server
-    // by how long we think one region takes opening.
-    Configuration conf = server.getConfiguration();
-    long perRegionOpenTimeGuesstimate =
-      conf.getLong("hbase.bulk.assignment.perregion.open.time", 1000);
-    int maxRegionsPerServer = 1;
-    for (List<HRegionInfo> regionList : bulkPlan.values()) {
-      int size = regionList.size();
-      if (size > maxRegionsPerServer) {
-        maxRegionsPerServer = size;
-      }
-    }
-    long timeout = perRegionOpenTimeGuesstimate * maxRegionsPerServer
-      + conf.getLong("hbase.regionserver.rpc.startup.waittime", 60000)
-      + conf.getLong("hbase.bulk.assignment.perregionserver.rpc.waittime",
-        30000) * bulkPlan.size();
-    LOG.debug("Timeout-on-RIT=" + timeout);
-    return timeout;
-  }
-
-  @Override
-  protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
-    return new UncaughtExceptionHandler() {
-      @Override
-      public void uncaughtException(Thread t, Throwable e) {
-        LOG.warn("Assigning regions in " + t.getName(), e);
-      }
-    };
-  }
-
-  private int reassignFailedPlans() {
-    List<HRegionInfo> reassigningRegions = new ArrayList<>();
-    for (Map.Entry<ServerName, List<HRegionInfo>> e : failedPlans.entrySet()) {
-      LOG.info("Failed assigning " + e.getValue().size()
-          + " regions to server " + e.getKey() + ", reassigning them");
-      reassigningRegions.addAll(failedPlans.remove(e.getKey()));
-    }
-    RegionStates regionStates = assignmentManager.getRegionStates();
-    for (HRegionInfo region : reassigningRegions) {
-      if (!regionStates.isRegionOnline(region)) {
-        assignmentManager.invokeAssign(region);
-      }
-    }
-    return reassigningRegions.size();
-  }
-
-  /**
-   * Manage bulk assigning to a server.
-   */
-  static class SingleServerBulkAssigner implements Runnable {
-    private final ServerName regionserver;
-    private final List<HRegionInfo> regions;
-    private final AssignmentManager assignmentManager;
-    private final Map<ServerName, List<HRegionInfo>> failedPlans;
-
-    SingleServerBulkAssigner(final ServerName regionserver,
-        final List<HRegionInfo> regions, final AssignmentManager am,
-        final Map<ServerName, List<HRegionInfo>> failedPlans) {
-      this.regionserver = regionserver;
-      this.regions = regions;
-      this.assignmentManager = am;
-      this.failedPlans = failedPlans;
-    }
-
-    @Override
-    public void run() {
-      try {
-       if (!assignmentManager.assign(regionserver, regions)) {
-         failedPlans.put(regionserver, regions);
-       }
-      } catch (Throwable t) {
-        LOG.warn("Failed bulking assigning " + regions.size()
-            + " region(s) to " + regionserver.getServerName()
-            + ", and continue to bulk assign others", t);
-        failedPlans.put(regionserver, regions);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index a1cbe53..9812aaf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -65,8 +65,6 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.RegionStateListener;
-import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -90,6 +88,10 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.master.balancer.BalancerChore;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
@@ -110,15 +112,14 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.DispatchMergingRegionsProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
-import org.apache.hadoop.hbase.master.procedure.MergeTableRegionsProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
-import org.apache.hadoop.hbase.master.procedure.SplitTableRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -332,7 +333,6 @@ public class HMaster extends HRegionServer implements MasterServices {
   private RegionNormalizerChore normalizerChore;
   private ClusterStatusChore clusterStatusChore;
   private ClusterStatusPublisher clusterStatusPublisherChore = null;
-  private PeriodicDoMetrics periodicDoMetricsChore = null;
 
   CatalogJanitor catalogJanitorChore;
   private ReplicationMetaCleaner replicationMetaCleaner;
@@ -431,19 +431,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  private static class PeriodicDoMetrics extends ScheduledChore {
-    private final HMaster server;
-    public PeriodicDoMetrics(int doMetricsInterval, final HMaster server) {
-      super(server.getServerName() + "-DoMetricsChore", server, doMetricsInterval);
-      this.server = server;
-    }
-
-    @Override
-    protected void chore() {
-      server.doMetrics();
-    }
-  }
-
   /**
    * Initializes the HMaster. The steps are as follows:
    * <p>
@@ -646,20 +633,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     return MasterDumpServlet.class;
   }
 
-  /**
-   * Emit the HMaster metrics, such as region in transition metrics.
-   * Surrounding in a try block just to be sure metrics doesn't abort HMaster.
-   */
-  private void doMetrics() {
-    try {
-      if (assignmentManager != null) {
-        assignmentManager.updateRegionsInTransitionMetrics();
-      }
-    } catch (Throwable e) {
-      LOG.error("Couldn't update metrics: " + e.getMessage());
-    }
-  }
-
   MetricsMaster getMasterMetrics() {
     return metricsMaster;
   }
@@ -682,8 +655,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
     this.splitOrMergeTracker.start();
 
-    this.assignmentManager = new AssignmentManager(this, serverManager,
-      this.balancer, this.service, this.metricsMaster, tableStateManager);
+    // Create Assignment Manager
+    this.assignmentManager = new AssignmentManager(this);
+    this.assignmentManager.start();
 
     this.replicationManager = new ReplicationManager(conf, zooKeeper, this);
 
@@ -867,10 +841,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.catalogJanitorChore = new CatalogJanitor(this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    // Do Metrics periodically
-    periodicDoMetricsChore = new PeriodicDoMetrics(msgInterval, this);
-    getChoreService().scheduleChore(periodicDoMetricsChore);
-
     status.setStatus("Starting cluster schema service");
     initClusterSchemaService();
 
@@ -883,7 +853,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     status.markComplete("Initialization successful");
-    LOG.info("Master has completed initialization");
+    LOG.info(String.format("Master has completed initialization %.3fsec",
+       (System.currentTimeMillis() - masterActiveTime) / 1000.0f));
     configurationManager.registerObserver(this.balancer);
 
     // Set master as 'initialized'.
@@ -961,8 +932,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Check zk for region servers that are up but didn't register
     for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
       // The isServerOnline check is opportunistic, correctness is handled inside
-      if (!this.serverManager.isServerOnline(sn)
-          && serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
+      if (!this.serverManager.isServerOnline(sn) &&
+          serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
         LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
       }
     }
@@ -976,14 +947,13 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   void initQuotaManager() throws IOException {
     MasterQuotaManager quotaManager = new MasterQuotaManager(this);
-    this.assignmentManager.setRegionStateListener((RegionStateListener)quotaManager);
+    this.assignmentManager.setRegionStateListener(quotaManager);
     quotaManager.start();
     this.quotaManager = quotaManager;
   }
 
   boolean isCatalogJanitorEnabled() {
-    return catalogJanitorChore != null ?
-      catalogJanitorChore.getEnabled() : false;
+    return catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
   }
 
   boolean isCleanerChoreEnabled() {
@@ -1091,7 +1061,6 @@ public class HMaster extends HRegionServer implements MasterServices {
   @Override
   protected void sendShutdownInterrupt() {
     super.sendShutdownInterrupt();
-    stopProcedureExecutor();
   }
 
   @Override
@@ -1116,15 +1085,20 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Stopping service threads");
     }
+
     // Clean up and close up shop
     if (this.logCleaner != null) this.logCleaner.cancel(true);
     if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
     if (this.replicationZKNodeCleanerChore != null) this.replicationZKNodeCleanerChore.cancel(true);
     if (this.replicationMetaCleaner != null) this.replicationMetaCleaner.cancel(true);
     if (this.quotaManager != null) this.quotaManager.stop();
+
     if (this.activeMasterManager != null) this.activeMasterManager.stop();
     if (this.serverManager != null) this.serverManager.stop();
     if (this.assignmentManager != null) this.assignmentManager.stop();
+
+    stopProcedureExecutor();
+
     if (this.walManager != null) this.walManager.stop();
     if (this.fileSystemManager != null) this.fileSystemManager.stop();
     if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
@@ -1150,16 +1124,20 @@ public class HMaster extends HRegionServer implements MasterServices {
         MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
     procedureStore.start(numThreads);
     procedureExecutor.start(numThreads, abortOnCorruption);
+    procEnv.getRemoteDispatcher().start();
   }
 
   private void stopProcedureExecutor() {
     if (procedureExecutor != null) {
       configurationManager.deregisterObserver(procedureExecutor.getEnvironment());
+      procedureExecutor.getEnvironment().getRemoteDispatcher().stop();
       procedureExecutor.stop();
+      procedureExecutor = null;
     }
 
     if (procedureStore != null) {
       procedureStore.stop(isAborted());
+      procedureStore = null;
     }
   }
 
@@ -1188,10 +1166,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (this.mobCompactThread != null) {
       this.mobCompactThread.close();
     }
-
-    if (this.periodicDoMetricsChore != null) {
-      periodicDoMetricsChore.cancel();
-    }
   }
 
   /**
@@ -1249,7 +1223,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Sleep to next balance plan start time
     // But if there are zero regions in transition, it can skip sleep to speed up.
     while (!interrupted && System.currentTimeMillis() < nextBalanceStartTime
-        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount() != 0) {
+        && this.assignmentManager.getRegionStates().hasRegionsInTransition()) {
       try {
         Thread.sleep(100);
       } catch (InterruptedException ie) {
@@ -1260,7 +1234,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Throttling by max number regions in transition
     while (!interrupted
         && maxRegionsInTransition > 0
-        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
+        && this.assignmentManager.getRegionStates().getRegionsInTransition().size()
         >= maxRegionsInTransition && System.currentTimeMillis() <= cutoffTime) {
       try {
         // sleep if the number of regions in transition exceeds the limit
@@ -1293,13 +1267,12 @@ public class HMaster extends HRegionServer implements MasterServices {
     synchronized (this.balancer) {
       // If balance not true, don't run balancer.
       if (!this.loadBalancerTracker.isBalancerOn()) return false;
-      // Only allow one balance run at at time.
-      if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
-        Set<RegionState> regionsInTransition =
-          this.assignmentManager.getRegionStates().getRegionsInTransition();
+        // Only allow one balance run at at time.
+      if (this.assignmentManager.hasRegionsInTransition()) {
+        List<RegionStateNode> regionsInTransition = assignmentManager.getRegionsInTransition();
         // if hbase:meta region is in transition, result of assignment cannot be recorded
         // ignore the force flag in that case
-        boolean metaInTransition = assignmentManager.getRegionStates().isMetaRegionInTransition();
+        boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
         String prefix = force && !metaInTransition ? "R" : "Not r";
         LOG.debug(prefix + "unning balancer because " + regionsInTransition.size() +
           " region(s) in transition: " + org.apache.commons.lang.StringUtils.
@@ -1332,7 +1305,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       //Give the balancer the current cluster state.
       this.balancer.setClusterStatus(getClusterStatus());
       this.balancer.setClusterLoad(
-              this.assignmentManager.getRegionStates().getAssignmentsByTable(true));
+              this.assignmentManager.getRegionStates().getAssignmentsByTable());
 
       for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
         List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
@@ -1351,7 +1324,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         for (RegionPlan plan: plans) {
           LOG.info("balance " + plan);
           //TODO: bulk assign
-          this.assignmentManager.balance(plan);
+          this.assignmentManager.moveAsync(plan);
           rpCount++;
 
           balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
@@ -1467,6 +1440,59 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
+  public long dispatchMergingRegions(
+      final HRegionInfo regionInfoA,
+      final HRegionInfo regionInfoB,
+      final boolean forcible,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    checkInitialized();
+
+    TableName tableName = regionInfoA.getTable();
+    if (tableName == null || regionInfoB.getTable() == null) {
+      throw new UnknownRegionException ("Can't merge regions without table associated");
+    }
+
+    if (!tableName.equals(regionInfoB.getTable())) {
+      throw new IOException ("Cannot merge regions from two different tables");
+    }
+
+    if (regionInfoA.compareTo(regionInfoB) == 0) {
+      throw new MergeRegionException(
+        "Cannot merge a region to itself " + regionInfoA + ", " + regionInfoB);
+    }
+
+    final HRegionInfo [] regionsToMerge = new HRegionInfo[2];
+    regionsToMerge [0] = regionInfoA;
+    regionsToMerge [1] = regionInfoB;
+
+    return MasterProcedureUtil.submitProcedure(
+        new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
+      @Override
+      protected void run() throws IOException {
+        MasterCoprocessorHost mcph = getMaster().getMasterCoprocessorHost();
+        if (mcph != null) {
+          mcph.preDispatchMerge(regionInfoA, regionInfoB);
+        }
+
+        LOG.info(getClientIdAuditPrefix() + " Dispatch merge regions " +
+          regionsToMerge[0].getEncodedName() + " and " + regionsToMerge[1].getEncodedName());
+
+        submitProcedure(new DispatchMergingRegionsProcedure(
+            procedureExecutor.getEnvironment(), tableName, regionsToMerge, forcible));
+        if (mcph != null) {
+          mcph.postDispatchMerge(regionInfoA, regionInfoB);
+        }
+      }
+
+      @Override
+      protected String getDescription() {
+        return "DispatchMergingRegionsProcedure";
+      }
+    });
+  }
+
+  @Override
   public long mergeRegions(
       final HRegionInfo[] regionsToMerge,
       final boolean forcible,
@@ -1515,24 +1541,19 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public long splitRegion(
-      final HRegionInfo regionInfo,
-      final byte[] splitRow,
-      final long nonceGroup,
-      final long nonce) throws IOException {
+  public long splitRegion(final HRegionInfo regionInfo, final byte[] splitRow,
+      final long nonceGroup, final long nonce)
+  throws IOException {
     checkInitialized();
-
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
       @Override
       protected void run() throws IOException {
         getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
-
-        LOG.info(getClientIdAuditPrefix() + " Split region " + regionInfo);
+        LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());
 
         // Execute the operation asynchronously
-        submitProcedure(new SplitTableRegionProcedure(procedureExecutor.getEnvironment(),
-            regionInfo, splitRow));
+        submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
       }
 
       @Override
@@ -1607,7 +1628,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       serverManager.sendRegionWarmup(rp.getDestination(), hri);
 
       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
-      this.assignmentManager.balance(rp);
+      this.assignmentManager.moveAsync(rp);
       if (this.cpHost != null) {
         this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
       }
@@ -2378,8 +2399,9 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     String clusterId = fileSystemManager != null ?
       fileSystemManager.getClusterId().toString() : null;
-    Set<RegionState> regionsInTransition = assignmentManager != null ?
-      assignmentManager.getRegionStates().getRegionsInTransition() : null;
+    List<RegionState> regionsInTransition = assignmentManager != null ?
+      assignmentManager.getRegionStates().getRegionsStateInTransition() : null;
+
     String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
     boolean balancerOn = loadBalancerTracker != null ?
       loadBalancerTracker.isBalancerOn() : false;
@@ -3114,6 +3136,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
    * @return The state of the switch
    */
+  @Override
   public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
     if (null == splitOrMergeTracker || isInMaintenanceMode()) {
       return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
index 277dcc8..0e86925 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/LoadBalancer.java
@@ -44,7 +44,9 @@ import edu.umd.cs.findbugs.annotations.Nullable;
  * <p>On cluster startup, bulk assignment can be used to determine
  * locations for all Regions in a cluster.
  *
- * <p>This classes produces plans for the {@link AssignmentManager} to execute.
+ * <p>This classes produces plans for the
+ * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager}
+ * to execute.
  */
 @InterfaceAudience.Private
 public interface LoadBalancer extends Configurable, Stoppable, ConfigurationObserver {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 8a7a387..534a507 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -789,6 +789,28 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preDispatchMerge(final HRegionInfo regionInfoA, final HRegionInfo regionInfoB)
+  throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.preDispatchMerge(ctx, regionInfoA, regionInfoB);
+      }
+    });
+  }
+
+  public void postDispatchMerge(final HRegionInfo regionInfoA, final HRegionInfo regionInfoB)
+  throws IOException {
+    execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
+      @Override
+      public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
+          throws IOException {
+        oserver.postDispatchMerge(ctx, regionInfoA, regionInfoB);
+      }
+    });
+  }
+
   public void preMergeRegions(final HRegionInfo[] regionsToMerge)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
index a921ab5..a48444c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
@@ -24,7 +24,6 @@ import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.util.Date;
 import java.util.Map;
-import java.util.Set;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
@@ -33,6 +32,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.monitoring.LogMonitoring;
 import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -117,9 +118,8 @@ public class MasterDumpServlet extends StateDumpServlet {
       return;
     }
 
-    Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
-    for (RegionState rs : regionsInTransition) {
-      String rid = rs.getRegion().getRegionNameAsString();
+    for (RegionStateNode rs : am.getRegionsInTransition()) {
+      String rid = rs.getRegionInfo().getEncodedName();
       out.println("Region " + rid + ": " + rs.toDescriptiveString());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index 1988e2d..fe93ce0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -151,7 +152,9 @@ public class MasterMetaBootstrap {
 
     // Work on meta region
     int assigned = 0;
-    long timeout = master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
+    // TODO: Unimplemented
+    // long timeout =
+    //   master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
       status.setStatus("Assigning hbase:meta region");
     } else {
@@ -160,37 +163,10 @@ public class MasterMetaBootstrap {
 
     // Get current meta state from zk.
     RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), replicaId);
-    HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
-        replicaId);
-    RegionStates regionStates = assignmentManager.getRegionStates();
-    regionStates.createRegionState(hri, metaState.getState(),
-        metaState.getServerName(), null);
-
-    if (!metaState.isOpened() || !master.getMetaTableLocator().verifyMetaRegionLocation(
-        master.getClusterConnection(), master.getZooKeeper(), timeout, replicaId)) {
-      ServerName currentMetaServer = metaState.getServerName();
-      if (master.getServerManager().isServerOnline(currentMetaServer)) {
-        if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
-          LOG.info("Meta was in transition on " + currentMetaServer);
-        } else {
-          LOG.info("Meta with replicaId " + replicaId + " was in transition on " +
-                    currentMetaServer);
-        }
-        assignmentManager.processRegionsInTransition(Collections.singletonList(metaState));
-      } else {
-        if (currentMetaServer != null) {
-          if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
-            splitMetaLogBeforeAssignment(currentMetaServer);
-            regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
-            previouslyFailedMetaRSs.add(currentMetaServer);
-          }
-        }
-        LOG.info("Re-assigning hbase:meta with replicaId, " + replicaId +
-            " it was on " + currentMetaServer);
-        assignmentManager.assignMeta(hri);
-      }
-      assigned++;
-    }
+    LOG.debug("meta state from zookeeper: " + metaState);
+    HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+      HRegionInfo.FIRST_META_REGIONINFO, replicaId);
+    assignmentManager.assignMeta(hri, metaState.getServerName());
 
     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
       // TODO: should we prevent from using state manager before meta was initialized?
@@ -206,7 +182,6 @@ public class MasterMetaBootstrap {
       master.getMasterWalManager().splitMetaLog(previouslyFailedMetaRSs);
     }
 
-    assignmentManager.setEnabledTable(TableName.META_TABLE_NAME);
     master.getTableStateManager().start();
 
     // Make sure a hbase:meta location is set. We need to enable SSH here since
@@ -214,7 +189,7 @@ public class MasterMetaBootstrap {
     // by SSH so that system tables can be assigned.
     // No need to wait for meta is assigned = 0 when meta is just verified.
     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(assigned != 0);
-    LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", location="
+    LOG.info("hbase:meta with replicaId " + replicaId + ", location="
       + master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper(), replicaId));
     status.setStatus("META assigned.");
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f86f800..6b13c12 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -32,9 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
@@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.ipc.PriorityFunction;
 import org.apache.hadoop.hbase.ipc.QosPriority;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
@@ -78,7 +78,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.*;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatRequest;
@@ -86,129 +85,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockH
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AddColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AssignRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.BalanceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DeleteTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DisableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DrainRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsInMaintenanceModeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsMasterRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSplitOrMergeEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListDrainingRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListProceduresResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyColumnResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ModifyTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.OfflineRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RemoveDrainFromRegionServersResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalogScanResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.*;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetCleanerChoreRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetQuotaResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SetSplitOrMergeEnabledResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.ShutdownResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.StopMasterResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -216,13 +94,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerRequest;
@@ -389,7 +264,11 @@ public class MasterRpcServices extends RSRpcServices
       ClusterStatusProtos.ServerLoad sl = request.getLoad();
       ServerName serverName = ProtobufUtil.toServerName(request.getServer());
       ServerLoad oldLoad = master.getServerManager().getLoad(serverName);
-      master.getServerManager().regionServerReport(serverName, new ServerLoad(sl));
+      ServerLoad newLoad = new ServerLoad(sl);
+      master.getServerManager().regionServerReport(serverName, newLoad);
+      int version = VersionInfoUtil.getCurrentClientVersionNumber();
+      master.getAssignmentManager().reportOnlineRegions(serverName,
+        version, newLoad.getRegionsLoad().keySet());
       if (sl != null && master.metricsMaster != null) {
         // Up our metrics.
         master.metricsMaster.incrementRequests(sl.getTotalNumberOfRequests()
@@ -462,25 +341,25 @@ public class MasterRpcServices extends RSRpcServices
   public AssignRegionResponse assignRegion(RpcController controller,
       AssignRegionRequest req) throws ServiceException {
     try {
-      final byte [] regionName = req.getRegion().getValue().toByteArray();
-      RegionSpecifierType type = req.getRegion().getType();
-      AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
-
       master.checkInitialized();
+
+      final RegionSpecifierType type = req.getRegion().getType();
       if (type != RegionSpecifierType.REGION_NAME) {
         LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
           + " actual: " + type);
       }
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      HRegionInfo regionInfo = regionStates.getRegionInfo(regionName);
-      if (regionInfo == null) throw new UnknownRegionException(Bytes.toString(regionName));
+
+      final byte[] regionName = req.getRegion().getValue().toByteArray();
+      final HRegionInfo regionInfo = master.getAssignmentManager().getRegionInfo(regionName);
+      if (regionInfo == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+
+      final AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
       if (master.cpHost != null) {
         if (master.cpHost.preAssign(regionInfo)) {
           return arr;
         }
       }
-      LOG.info(master.getClientIdAuditPrefix()
-        + " assign " + regionInfo.getRegionNameAsString());
+      LOG.info(master.getClientIdAuditPrefix() + " assign " + regionInfo.getRegionNameAsString());
       master.getAssignmentManager().assign(regionInfo, true);
       if (master.cpHost != null) {
         master.cpHost.postAssign(regionInfo);
@@ -491,6 +370,7 @@ public class MasterRpcServices extends RSRpcServices
     }
   }
 
+
   @Override
   public BalanceResponse balance(RpcController controller,
       BalanceRequest request) throws ServiceException {
@@ -710,8 +590,7 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
-  public SplitTableRegionResponse splitRegion(
-      final RpcController controller,
+  public SplitTableRegionResponse splitRegion(final RpcController controller,
       final SplitTableRegionRequest request) throws ServiceException {
     try {
       long procId = master.splitRegion(
@@ -1281,24 +1160,24 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public OfflineRegionResponse offlineRegion(RpcController controller,
       OfflineRegionRequest request) throws ServiceException {
-    final byte [] regionName = request.getRegion().getValue().toByteArray();
-    RegionSpecifierType type = request.getRegion().getType();
-    if (type != RegionSpecifierType.REGION_NAME) {
-      LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
-        + " actual: " + type);
-    }
-
     try {
       master.checkInitialized();
-      Pair<HRegionInfo, ServerName> pair =
-        MetaTableAccessor.getRegion(master.getConnection(), regionName);
-      if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
-      HRegionInfo hri = pair.getFirst();
+
+      final RegionSpecifierType type = request.getRegion().getType();
+      if (type != RegionSpecifierType.REGION_NAME) {
+        LOG.warn("moveRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
+          + " actual: " + type);
+      }
+
+      final byte[] regionName = request.getRegion().getValue().toByteArray();
+      final HRegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
+      if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+
       if (master.cpHost != null) {
         master.cpHost.preRegionOffline(hri);
       }
       LOG.info(master.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
-      master.getAssignmentManager().regionOffline(hri);
+      master.getAssignmentManager().offlineRegion(hri);
       if (master.cpHost != null) {
         master.cpHost.postRegionOffline(hri);
       }
@@ -1439,28 +1318,18 @@ public class MasterRpcServices extends RSRpcServices
   public UnassignRegionResponse unassignRegion(RpcController controller,
       UnassignRegionRequest req) throws ServiceException {
     try {
-      final byte [] regionName = req.getRegion().getValue().toByteArray();
-      RegionSpecifierType type = req.getRegion().getType();
-      final boolean force = req.getForce();
-      UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
-
       master.checkInitialized();
+      final RegionSpecifierType type = req.getRegion().getType();
       if (type != RegionSpecifierType.REGION_NAME) {
         LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
           + " actual: " + type);
       }
-      Pair<HRegionInfo, ServerName> pair =
-        MetaTableAccessor.getRegion(master.getConnection(), regionName);
-      if (Bytes.equals(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(),regionName)) {
-        pair = new Pair<>(HRegionInfo.FIRST_META_REGIONINFO,
-            master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper()));
-      }
-      if (pair == null) {
-        throw new UnknownRegionException(Bytes.toString(regionName));
-      }
+      final byte[] regionName = req.getRegion().getValue().toByteArray();
+      final HRegionInfo hri = master.getAssignmentManager().getRegionInfo(regionName);
+      if (hri == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
 
-      if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
-      HRegionInfo hri = pair.getFirst();
+      final boolean force = req.getForce();
+      final UnassignRegionResponse urr = UnassignRegionResponse.newBuilder().build();
       if (master.cpHost != null) {
         if (master.cpHost.preUnassign(hri, force)) {
           return urr;
@@ -1472,7 +1341,6 @@ public class MasterRpcServices extends RSRpcServices
       if (master.cpHost != null) {
         master.cpHost.postUnassign(hri, force);
       }
-
       return urr;
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
@@ -1484,26 +1352,7 @@ public class MasterRpcServices extends RSRpcServices
       ReportRegionStateTransitionRequest req) throws ServiceException {
     try {
       master.checkServiceStarted();
-      RegionStateTransition rt = req.getTransition(0);
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      for (RegionInfo ri : rt.getRegionInfoList())  {
-        TableName tableName = ProtobufUtil.toTableName(ri.getTableName());
-        if (!(TableName.META_TABLE_NAME.equals(tableName)
-            && regionStates.getRegionState(HRegionInfo.FIRST_META_REGIONINFO) != null)
-              && !master.getAssignmentManager().isFailoverCleanupDone()) {
-          // Meta region is assigned before master finishes the
-          // failover cleanup. So no need this check for it
-          throw new PleaseHoldException("Master is rebuilding user regions");
-        }
-      }
-      ServerName sn = ProtobufUtil.toServerName(req.getServer());
-      String error = master.getAssignmentManager().onRegionTransition(sn, rt);
-      ReportRegionStateTransitionResponse.Builder rrtr =
-        ReportRegionStateTransitionResponse.newBuilder();
-      if (error != null) {
-        rrtr.setErrorMessage(error);
-      }
-      return rrtr.build();
+      return master.getAssignmentManager().reportRegionStateTransition(req);
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -2006,4 +1855,34 @@ public class MasterRpcServices extends RSRpcServices
       throw new ServiceException(e);
     }
   }
-}
+
+  @Override
+  public DispatchMergingRegionsResponse dispatchMergingRegions(RpcController controller,
+      DispatchMergingRegionsRequest request) throws ServiceException {
+    final byte[] encodedNameOfRegionA = request.getRegionA().getValue().toByteArray();
+    final byte[] encodedNameOfRegionB = request.getRegionB().getValue().toByteArray();
+    if (request.getRegionA().getType() != RegionSpecifierType.ENCODED_REGION_NAME ||
+        request.getRegionB().getType() != RegionSpecifierType.ENCODED_REGION_NAME) {
+      LOG.warn("mergeRegions specifier type: expected: " + RegionSpecifierType.ENCODED_REGION_NAME +
+          " actual: region_a=" +
+          request.getRegionA().getType() + ", region_b=" +
+          request.getRegionB().getType());
+    }
+    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+    RegionState regionStateA = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionA));
+    RegionState regionStateB = regionStates.getRegionState(Bytes.toString(encodedNameOfRegionB));
+    if (regionStateA == null || regionStateB == null) {
+      throw new ServiceException(new UnknownRegionException(
+        Bytes.toStringBinary(regionStateA == null? encodedNameOfRegionA: encodedNameOfRegionB)));
+    }
+    final HRegionInfo regionInfoA = regionStateA.getRegion();
+    final HRegionInfo regionInfoB = regionStateB.getRegion();
+    try {
+      long procId = master.dispatchMergingRegions(regionInfoA, regionInfoB, request.getForcible(),
+          request.getNonceGroup(), request.getNonce());
+      return DispatchMergingRegionsResponse.newBuilder().setProcId(procId).build();
+    } catch (IOException ioe) {
+      throw new ServiceException(ioe);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 66758f8..5a45fcf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
@@ -266,6 +268,23 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
+   * Merge two regions. The real implementation is on the regionserver, master
+   * just move the regions together and send MERGE RPC to regionserver
+   * @param region_a region to merge
+   * @param region_b region to merge
+   * @param forcible true if do a compulsory merge, otherwise we will only merge
+   *          two adjacent regions
+   * @return procedure Id
+   * @throws IOException
+   */
+  long dispatchMergingRegions(
+    final HRegionInfo region_a,
+    final HRegionInfo region_b,
+    final boolean forcible,
+    final long nonceGroup,
+    final long nonce) throws IOException;
+
+  /**
    * Merge regions in a table.
    * @param regionsToMerge daughter regions to merge
    * @param forcible whether to force to merge even two regions are not adjacent
@@ -394,6 +413,8 @@ public interface MasterServices extends Server {
    */
   boolean isStopping();
 
+  boolean isSplitOrMergeEnabled(MasterSwitchType switchType);
+
   /**
    * @return Favored Nodes Manager
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index 27aca94..9ac5ba2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -218,9 +219,7 @@ public class MasterWalManager {
   }
 
   public void splitLog(final ServerName serverName) throws IOException {
-    Set<ServerName> serverNames = new HashSet<>();
-    serverNames.add(serverName);
-    splitLog(serverNames);
+    splitLog(Collections.<ServerName>singleton(serverName));
   }
 
   /**
@@ -228,9 +227,7 @@ public class MasterWalManager {
    * @param serverName logs belonging to this server will be split
    */
   public void splitMetaLog(final ServerName serverName) throws IOException {
-    Set<ServerName> serverNames = new HashSet<>();
-    serverNames.add(serverName);
-    splitMetaLog(serverNames);
+    splitMetaLog(Collections.<ServerName>singleton(serverName));
   }
 
   /**
@@ -342,4 +339,4 @@ public class MasterWalManager {
   public RecoveryMode getLogRecoveryMode() {
     return this.splitLogManager.getRecoveryMode();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
index 40e79ae..c7ce9a9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManager.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.master;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 
 public class MetricsAssignmentManager {
-
   private final MetricsAssignmentManagerSource assignmentManagerSource;
 
   public MetricsAssignmentManager() {
@@ -33,19 +32,11 @@ public class MetricsAssignmentManager {
     return assignmentManagerSource;
   }
 
-  public void updateAssignmentTime(long time) {
-    assignmentManagerSource.updateAssignmentTime(time);
-  }
-
-  public void updateBulkAssignTime(long time) {
-    assignmentManagerSource.updateBulkAssignTime(time);
-  }
-
   /**
    * set new value for number of regions in transition.
    * @param ritCount
    */
-  public void updateRITCount(int ritCount) {
+  public void updateRITCount(final int ritCount) {
     assignmentManagerSource.setRIT(ritCount);
   }
 
@@ -54,14 +45,15 @@ public class MetricsAssignmentManager {
    * as defined by the property rit.metrics.threshold.time.
    * @param ritCountOverThreshold
    */
-  public void updateRITCountOverThreshold(int ritCountOverThreshold) {
+  public void updateRITCountOverThreshold(final int ritCountOverThreshold) {
     assignmentManagerSource.setRITCountOverThreshold(ritCountOverThreshold);
   }
+
   /**
    * update the timestamp for oldest region in transition metrics.
    * @param timestamp
    */
-  public void updateRITOldestAge(long timestamp) {
+  public void updateRITOldestAge(final long timestamp) {
     assignmentManagerSource.setRITOldestAge(timestamp);
   }
 
@@ -72,4 +64,27 @@ public class MetricsAssignmentManager {
   public void updateRitDuration(long duration) {
     assignmentManagerSource.updateRitDuration(duration);
   }
+
+  /*
+   * Increment the count of assignment operation (assign/unassign).
+   */
+  public void incrementOperationCounter() {
+    assignmentManagerSource.incrementOperationCounter();
+  }
+
+  /**
+   * Add the time took to perform the last assign operation
+   * @param time
+   */
+  public void updateAssignTime(final long time) {
+    assignmentManagerSource.updateAssignTime(time);
+  }
+
+  /**
+   * Add the time took to perform the last unassign operation
+   * @param time
+   */
+  public void updateUnassignTime(final long time) {
+    assignmentManagerSource.updateUnassignTime(time);
+  }
 }


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
deleted file mode 100644
index 449e1e6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ /dev/null
@@ -1,1402 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
-import org.apache.hadoop.hbase.ServerLoad;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.UnknownRegionException;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
-import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-/**
- * This tests AssignmentManager with a testing cluster.
- */
-@SuppressWarnings("deprecation")
-@Category({MasterTests.class, MediumTests.class})
-public class TestAssignmentManagerOnCluster {
-  private final static byte[] FAMILY = Bytes.toBytes("FAMILY");
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  final static Configuration conf = TEST_UTIL.getConfiguration();
-  private static Admin admin;
-
-  @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    // Using the our load balancer to control region plans
-    conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
-      MyLoadBalancer.class, LoadBalancer.class);
-    conf.setClass(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-      MyRegionObserver.class, RegionObserver.class);
-    // Reduce the maximum attempts to speed up the test
-    conf.setInt("hbase.assignment.maximum.attempts", 3);
-    conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
-    conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
-
-    TEST_UTIL.startMiniCluster(1, 4, null, MyMaster.class, MyRegionServer.class);
-    admin = TEST_UTIL.getAdmin();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  /**
-   * This tests restarting meta regionserver
-   */
-  @Test (timeout=180000)
-  public void testRestartMetaRegionServer() throws Exception {
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    boolean stoppedARegionServer = false;
-    try {
-      HMaster master = cluster.getMaster();
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      ServerName metaServerName = regionStates.getRegionServerOfRegion(
-        HRegionInfo.FIRST_META_REGIONINFO);
-      if (master.getServerName().equals(metaServerName)) {
-        // Move meta off master
-        metaServerName = cluster.getLiveRegionServerThreads()
-          .get(0).getRegionServer().getServerName();
-        master.move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
-                Bytes.toBytes(metaServerName.getServerName()));
-        TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-      }
-      RegionState metaState =
-        MetaTableLocator.getMetaRegionState(master.getZooKeeper());
-      assertEquals("Meta should be not in transition",
-          metaState.getState(), RegionState.State.OPEN);
-      assertNotEquals("Meta should be moved off master",
-        metaState.getServerName(), master.getServerName());
-      assertEquals("Meta should be on the meta server",
-        metaState.getServerName(), metaServerName);
-      cluster.killRegionServer(metaServerName);
-      stoppedARegionServer = true;
-      cluster.waitForRegionServerToStop(metaServerName, 60000);
-
-      // Wait for SSH to finish
-      final ServerName oldServerName = metaServerName;
-      final ServerManager serverManager = master.getServerManager();
-      TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
-        @Override
-        public boolean evaluate() throws Exception {
-          return serverManager.isServerDead(oldServerName)
-            && !serverManager.areDeadServersInProgress();
-        }
-      });
-
-      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-      // Now, make sure meta is assigned
-      assertTrue("Meta should be assigned",
-        regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
-      // Now, make sure meta is registered in zk
-      metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
-      assertEquals("Meta should be not in transition",
-          metaState.getState(), RegionState.State.OPEN);
-      assertEquals("Meta should be assigned", metaState.getServerName(),
-        regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO));
-      assertNotEquals("Meta should be assigned on a different server",
-        metaState.getServerName(), metaServerName);
-    } finally {
-      if (stoppedARegionServer) {
-        cluster.startRegionServer();
-      }
-    }
-  }
-
-  /**
-   * This tests region assignment
-   */
-  @Test (timeout=60000)
-  public void testAssignRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      TEST_UTIL.assignRegion(hri);
-
-      RegionStates regionStates = am.getRegionStates();
-      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-
-      // Region is assigned now. Let's assign it again.
-      // Master should not abort, and region should be assigned.
-      TEST_UTIL.getAdmin().assign(hri.getRegionName());
-      master.getAssignmentManager().waitForAssignment(hri);
-      RegionState newState = regionStates.getRegionState(hri);
-      assertTrue(newState.isOpened());
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests region assignment on a simulated restarted server
-   */
-  @Test (timeout=120000)
-  public void testAssignRegionOnRestartedServer() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
-    TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
-    //restart the master so that conf take into affect
-    TEST_UTIL.getMiniHBaseCluster().startMaster();
-
-    ServerName deadServer = null;
-    HMaster master = null;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      final HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      master = TEST_UTIL.getHBaseCluster().getMaster();
-      Set<ServerName> onlineServers = master.getServerManager().getOnlineServers().keySet();
-      assertFalse("There should be some servers online", onlineServers.isEmpty());
-
-      // Use the first server as the destination server
-      ServerName destServer = onlineServers.iterator().next();
-
-      // Created faked dead server that is still online in master
-      deadServer = ServerName.valueOf(destServer.getHostname(),
-          destServer.getPort(), destServer.getStartcode() - 100L);
-      master.getServerManager().recordNewServerWithLock(deadServer, ServerLoad.EMPTY_SERVERLOAD);
-
-      final AssignmentManager am = master.getAssignmentManager();
-      RegionPlan plan = new RegionPlan(hri, null, deadServer);
-      am.addPlan(hri.getEncodedName(), plan);
-      TEST_UTIL.assignRegion(hri);
-
-      TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
-        @Override
-        public boolean evaluate() throws Exception {
-          return ! am.getRegionStates().isRegionInTransition(hri);
-        }
-      });
-
-    assertFalse("Region should be assigned", am.getRegionStates().isRegionInTransition(hri));
-    } finally {
-      if (deadServer != null) {
-        master.getServerManager().expireServer(deadServer);
-      }
-
-      TEST_UTIL.deleteTable(tableName);
-
-      // reset the value for other tests
-      TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 3);
-      ServerName masterServerName = TEST_UTIL.getMiniHBaseCluster().getMaster().getServerName();
-      TEST_UTIL.getMiniHBaseCluster().stopMaster(masterServerName);
-      TEST_UTIL.getMiniHBaseCluster().startMaster();
-    }
-  }
-
-  /**
-   * This tests offlining a region
-   */
-  @Test (timeout=60000)
-  public void testOfflineRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HRegionInfo hri = createTableAndGetOneRegion(tableName);
-
-      RegionStates regionStates = TEST_UTIL.getHBaseCluster().
-        getMaster().getAssignmentManager().getRegionStates();
-      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-      admin.offline(hri.getRegionName());
-
-      long timeoutTime = System.currentTimeMillis() + 800;
-      while (true) {
-        if (regionStates.getRegionByStateOfTable(tableName)
-            .get(RegionState.State.OFFLINE).contains(hri))
-          break;
-        long now = System.currentTimeMillis();
-        if (now > timeoutTime) {
-          fail("Failed to offline the region in time");
-          break;
-        }
-        Thread.sleep(10);
-      }
-      RegionState regionState = regionStates.getRegionState(hri);
-      assertTrue(regionState.isOffline());
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests moving a region
-   */
-  @Test (timeout=50000)
-  public void testMoveRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HRegionInfo hri = createTableAndGetOneRegion(tableName);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
-      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-      ServerManager serverManager = master.getServerManager();
-      ServerName destServerName = null;
-      List<JVMClusterUtil.RegionServerThread> regionServers =
-        TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads();
-      for (JVMClusterUtil.RegionServerThread regionServer: regionServers) {
-        HRegionServer destServer = regionServer.getRegionServer();
-        destServerName = destServer.getServerName();
-        if (!destServerName.equals(serverName)
-            && serverManager.isServerOnline(destServerName)) {
-          break;
-        }
-      }
-      assertTrue(destServerName != null
-        && !destServerName.equals(serverName));
-      TEST_UTIL.getAdmin().move(hri.getEncodedNameAsBytes(),
-        Bytes.toBytes(destServerName.getServerName()));
-
-      long timeoutTime = System.currentTimeMillis() + 30000;
-      while (true) {
-        ServerName sn = regionStates.getRegionServerOfRegion(hri);
-        if (sn != null && sn.equals(destServerName)) {
-          TEST_UTIL.assertRegionOnServer(hri, sn, 200);
-          break;
-        }
-        long now = System.currentTimeMillis();
-        if (now > timeoutTime) {
-          fail("Failed to move the region in time: "
-            + regionStates.getRegionState(hri));
-        }
-        regionStates.waitForUpdate(50);
-      }
-
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * If a table is deleted, we should not be able to move it anymore.
-   * Otherwise, the region will be brought back.
-   * @throws Exception
-   */
-  @Test (timeout=50000)
-  public void testMoveRegionOfDeletedTable() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    Admin admin = TEST_UTIL.getAdmin();
-    try {
-      HRegionInfo hri = createTableAndGetOneRegion(tableName);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      RegionStates regionStates = am.getRegionStates();
-      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-      ServerName destServerName = null;
-      for (int i = 0; i < 3; i++) {
-        HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
-        if (!destServer.getServerName().equals(serverName)) {
-          destServerName = destServer.getServerName();
-          break;
-        }
-      }
-      assertTrue(destServerName != null
-        && !destServerName.equals(serverName));
-
-      TEST_UTIL.deleteTable(tableName);
-
-      try {
-        admin.move(hri.getEncodedNameAsBytes(),
-          Bytes.toBytes(destServerName.getServerName()));
-        fail("We should not find the region");
-      } catch (IOException ioe) {
-        assertTrue(ioe instanceof UnknownRegionException);
-      }
-
-      am.balance(new RegionPlan(hri, serverName, destServerName));
-      assertFalse("The region should not be in transition",
-        regionStates.isRegionInTransition(hri));
-    } finally {
-      if (admin.tableExists(tableName)) {
-        TEST_UTIL.deleteTable(tableName);
-      }
-    }
-  }
-
-  HRegionInfo createTableAndGetOneRegion(
-      final TableName tableName) throws IOException, InterruptedException {
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(FAMILY));
-    admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5);
-
-    // wait till the table is assigned
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    long timeoutTime = System.currentTimeMillis() + 1000;
-    while (true) {
-      List<HRegionInfo> regions = master.getAssignmentManager().
-        getRegionStates().getRegionsOfTable(tableName);
-      if (regions.size() > 3) {
-        return regions.get(2);
-      }
-      long now = System.currentTimeMillis();
-      if (now > timeoutTime) {
-        fail("Could not find an online region");
-      }
-      Thread.sleep(10);
-    }
-  }
-
-  /**
-   * This tests assign a region while it's closing.
-   */
-  @Test (timeout=60000)
-  public void testAssignWhileClosing() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      assertTrue(TEST_UTIL.assignRegion(hri));
-
-      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
-      MyRegionObserver.preCloseEnabled.set(true);
-      am.unassign(hri);
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_CLOSE, state.getState());
-
-      MyRegionObserver.preCloseEnabled.set(false);
-      am.unassign(hri);
-
-      // region is closing now, will be re-assigned automatically.
-      // now, let's forcefully assign it again. it should be
-      // assigned properly and no double-assignment
-      am.assign(hri, true);
-
-      // let's check if it's assigned after it's out of transition
-      am.waitOnRegionToClearRegionsInTransition(hri);
-      assertTrue(am.waitForAssignment(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
-    } finally {
-      MyRegionObserver.preCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests region close failed
-   */
-  @Test (timeout=60000)
-  public void testCloseFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      assertTrue(TEST_UTIL.assignRegion(hri));
-      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
-
-      MyRegionObserver.preCloseEnabled.set(true);
-      am.unassign(hri);
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_CLOSE, state.getState());
-
-      MyRegionObserver.preCloseEnabled.set(false);
-      am.unassign(hri);
-
-      // region may still be assigned now since it's closing,
-      // let's check if it's assigned after it's out of transition
-      am.waitOnRegionToClearRegionsInTransition(hri);
-
-      // region should be closed and re-assigned
-      assertTrue(am.waitForAssignment(hri));
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-    } finally {
-      MyRegionObserver.preCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests region open failed
-   */
-  @Test (timeout=60000)
-  public void testOpenFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      MyLoadBalancer.controledRegion = hri;
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      assertFalse(TEST_UTIL.assignRegion(hri));
-
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
-      // Failed to open since no plan, so it's on no server
-      assertNull(state.getServerName());
-
-      MyLoadBalancer.controledRegion = null;
-      assertTrue(TEST_UTIL.assignRegion(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-    } finally {
-      MyLoadBalancer.controledRegion = null;
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests round-robin assignment failed due to no bulkplan
-   */
-  @Test (timeout=60000)
-  public void testRoundRobinAssignmentFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = admin.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-
-      // round-robin assignment but balancer cannot find a plan
-      // assignment should fail
-      MyLoadBalancer.controledRegion = hri;
-      // if bulk assignment cannot update region state to online
-      // or failed_open this waits until timeout
-      assertFalse(TEST_UTIL.assignRegion(hri));
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
-      // Failed to open since no plan, so it's on no server
-      assertNull(state.getServerName());
-
-      // try again with valid plan
-      MyLoadBalancer.controledRegion = null;
-      assertTrue(TEST_UTIL.assignRegion(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-    } finally {
-      MyLoadBalancer.controledRegion = null;
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests retain assignment failed due to no bulkplan
-   */
-  @Test (timeout=60000)
-  public void testRetainAssignmentFailed() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      MyLoadBalancer.controledRegion = hri;
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-
-      Map<HRegionInfo, ServerName> regions = new HashMap<>();
-      ServerName dest = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
-      regions.put(hri, dest);
-      // retainAssignment but balancer cannot find a plan
-      // assignment should fail
-      am.assign(regions);
-
-      // if retain assignment cannot update region state to online
-      // or failed_open this waits until timeout
-      assertFalse(am.waitForAssignment(hri));
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
-      // Failed to open since no plan, so it's on no server
-      assertNull(state.getServerName());
-
-      // try retainAssigment again with valid plan
-      MyLoadBalancer.controledRegion = null;
-      am.assign(regions);
-      assertTrue(am.waitForAssignment(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-
-      // it retains on same server as specified
-      assertEquals(serverName, dest);
-    } finally {
-      MyLoadBalancer.controledRegion = null;
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests region open failure which is not recoverable
-   */
-  @Test (timeout=60000)
-  public void testOpenFailedUnrecoverable() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      FileSystem fs = FileSystem.get(conf);
-      Path tableDir= FSUtils.getTableDir(FSUtils.getRootDir(conf), tableName);
-      Path regionDir = new Path(tableDir, hri.getEncodedName());
-      // create a file named the same as the region dir to
-      // mess up with region opening
-      fs.create(regionDir, true);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      assertFalse(TEST_UTIL.assignRegion(hri));
-
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      assertEquals(RegionState.State.FAILED_OPEN, state.getState());
-      // Failed to open due to file system issue. Region state should
-      // carry the opening region server so that we can force close it
-      // later on before opening it again. See HBASE-9092.
-      assertNotNull(state.getServerName());
-
-      // remove the blocking file, so that region can be opened
-      fs.delete(regionDir, true);
-      assertTrue(TEST_UTIL.assignRegion(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  @Test (timeout=60000)
-  public void testSSHWhenDisablingTableRegionsInOpeningOrPendingOpenState() throws Exception {
-    final TableName table =
-        TableName.valueOf
-            ("testSSHWhenDisablingTableRegionsInOpeningOrPendingOpenState");
-    AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
-    HRegionInfo hri = null;
-    ServerName serverName = null;
-    try {
-      hri = createTableAndGetOneRegion(table);
-      serverName = am.getRegionStates().getRegionServerOfRegion(hri);
-      ServerName destServerName = null;
-      HRegionServer destServer = null;
-      for (int i = 0; i < 3; i++) {
-        destServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
-        if (!destServer.getServerName().equals(serverName)) {
-          destServerName = destServer.getServerName();
-          break;
-        }
-      }
-      am.regionOffline(hri);
-      am.getRegionStates().updateRegionState(hri, RegionState.State.PENDING_OPEN, destServerName);
-
-      am.getTableStateManager().setTableState(table, TableState.State.DISABLING);
-      List<HRegionInfo> toAssignRegions = am.cleanOutCrashedServerReferences(destServerName);
-      assertTrue("Regions to be assigned should be empty.", toAssignRegions.isEmpty());
-      assertTrue("Regions to be assigned should be empty.", am.getRegionStates()
-          .getRegionState(hri).isOffline());
-    } finally {
-      if (hri != null && serverName != null) {
-        am.regionOnline(hri, serverName);
-      }
-      am.getTableStateManager().setTableState(table, TableState.State.ENABLED);
-      TEST_UTIL.getAdmin().disableTable(table);
-      TEST_UTIL.deleteTable(table);
-    }
-  }
-
-  /**
-   * This tests region close hanging
-   */
-  @Test (timeout=60000)
-  public void testCloseHang() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      assertTrue(TEST_UTIL.assignRegion(hri));
-      ServerName sn = am.getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, sn, 6000);
-
-      MyRegionObserver.postCloseEnabled.set(true);
-      am.unassign(hri);
-
-      // Let region closing move ahead. The region should be closed
-      // properly and re-assigned automatically
-      MyRegionObserver.postCloseEnabled.set(false);
-
-      // region may still be assigned now since it's closing,
-      // let's check if it's assigned after it's out of transition
-      am.waitOnRegionToClearRegionsInTransition(hri);
-
-      // region should be closed and re-assigned
-      assertTrue(am.waitForAssignment(hri));
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-    } finally {
-      MyRegionObserver.postCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * This tests region close racing with open
-   */
-  @Test (timeout=60000)
-  public void testOpenCloseRacing() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-      meta.close();
-
-      MyRegionObserver.postOpenEnabled.set(true);
-      MyRegionObserver.postOpenCalled = false;
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      // Region will be opened, but it won't complete
-      am.assign(hri);
-      long end = EnvironmentEdgeManager.currentTime() + 20000;
-      // Wait till postOpen is called
-      while (!MyRegionObserver.postOpenCalled ) {
-        assertFalse("Timed out waiting for postOpen to be called",
-          EnvironmentEdgeManager.currentTime() > end);
-        Thread.sleep(300);
-      }
-
-      // Now let's unassign it, it should do nothing
-      am.unassign(hri);
-      RegionState state = am.getRegionStates().getRegionState(hri);
-      ServerName oldServerName = state.getServerName();
-      assertTrue(state.isOpening() && oldServerName != null);
-
-      // Now the region is stuck in opening
-      // Let's forcefully re-assign it to trigger closing/opening
-      // racing. This test is to make sure this scenario
-      // is handled properly.
-      MyRegionObserver.postOpenEnabled.set(false);
-      ServerName destServerName = null;
-      int numRS = TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size();
-      for (int i = 0; i < numRS; i++) {
-        HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(i);
-        if (!destServer.getServerName().equals(oldServerName)) {
-          destServerName = destServer.getServerName();
-          break;
-        }
-      }
-      assertNotNull(destServerName);
-      assertFalse("Region should be assigned on a new region server",
-        oldServerName.equals(destServerName));
-      List<HRegionInfo> regions = new ArrayList<>();
-      regions.add(hri);
-      am.assign(destServerName, regions);
-
-      // let's check if it's assigned after it's out of transition
-      am.waitOnRegionToClearRegionsInTransition(hri);
-      assertTrue(am.waitForAssignment(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
-    } finally {
-      MyRegionObserver.postOpenEnabled.set(false);
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * Test force unassign/assign a region hosted on a dead server
-   */
-  @Test (timeout=60000)
-  public void testAssignRacingWithSSH() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    MyMaster master = null;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      // Assign the region
-      master = (MyMaster)cluster.getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-
-      am.assign(hri);
-
-      // Hold SSH before killing the hosting server
-      master.enableSSH(false);
-
-
-      RegionStates regionStates = am.getRegionStates();
-      ServerName metaServer = regionStates.getRegionServerOfRegion(
-        HRegionInfo.FIRST_META_REGIONINFO);
-      while (true) {
-        assertTrue(am.waitForAssignment(hri));
-        RegionState state = regionStates.getRegionState(hri);
-        ServerName oldServerName = state.getServerName();
-        if (!ServerName.isSameHostnameAndPort(oldServerName, metaServer)) {
-          // Kill the hosting server, which doesn't have meta on it.
-          cluster.killRegionServer(oldServerName);
-          cluster.waitForRegionServerToStop(oldServerName, -1);
-          break;
-        }
-        int i = cluster.getServerWithMeta();
-        HRegionServer rs = cluster.getRegionServer(i == 0 ? 1 : 0);
-        oldServerName = rs.getServerName();
-        master.move(hri.getEncodedNameAsBytes(),
-          Bytes.toBytes(oldServerName.getServerName()));
-      }
-
-      // You can't assign a dead region before SSH
-      am.assign(hri, true);
-      RegionState state = regionStates.getRegionState(hri);
-      assertTrue(state.isFailedClose());
-
-      // You can't unassign a dead region before SSH either
-      am.unassign(hri);
-      state = regionStates.getRegionState(hri);
-      assertTrue(state.isFailedClose());
-
-      // Enable SSH so that log can be split
-      master.enableSSH(true);
-
-      // let's check if it's assigned after it's out of transition.
-      // no need to assign it manually, SSH should do it
-      am.waitOnRegionToClearRegionsInTransition(hri);
-      assertTrue(am.waitForAssignment(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
-    } finally {
-      if (master != null) {
-        master.enableSSH(true);
-      }
-      TEST_UTIL.deleteTable(tableName);
-      cluster.startRegionServer();
-    }
-  }
-
-  /**
-   * Test SSH waiting for extra region server for assignment
-   */
-  @Test (timeout=300000)
-  public void testSSHWaitForServerToAssignRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    boolean startAServer = false;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      HMaster master = cluster.getMaster();
-      final ServerManager serverManager = master.getServerManager();
-      MyLoadBalancer.countRegionServers = Integer.valueOf(
-        serverManager.countOfRegionServers());
-      HRegionServer rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
-      assertNotNull("First region should be assigned", rs);
-      final ServerName serverName = rs.getServerName();
-      // Wait till SSH tried to assign regions a several times
-      int counter = MyLoadBalancer.counter.get() + 5;
-      cluster.killRegionServer(serverName);
-      startAServer = true;
-      cluster.waitForRegionServerToStop(serverName, -1);
-      while (counter > MyLoadBalancer.counter.get()) {
-        Thread.sleep(1000);
-      }
-      cluster.startRegionServer();
-      startAServer = false;
-      // Wait till the dead server is processed by SSH
-      TEST_UTIL.waitFor(120000, 1000, new Waiter.Predicate<Exception>() {
-        @Override
-        public boolean evaluate() throws Exception {
-          return serverManager.isServerDead(serverName)
-            && !serverManager.areDeadServersInProgress();
-        }
-      });
-      TEST_UTIL.waitUntilNoRegionsInTransition(300000);
-
-      rs = TEST_UTIL.getRSForFirstRegionInTable(tableName);
-      assertTrue("First region should be re-assigned to a different server",
-        rs != null && !serverName.equals(rs.getServerName()));
-    } finally {
-      MyLoadBalancer.countRegionServers = null;
-      TEST_UTIL.deleteTable(tableName);
-      if (startAServer) {
-        cluster.startRegionServer();
-      }
-    }
-  }
-
-  /**
-   * Test force unassign/assign a region of a disabled table
-   */
-  @Test (timeout=60000)
-  public void testAssignDisabledRegion() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    MyMaster master = null;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      // Assign the region
-      master = (MyMaster)cluster.getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      RegionStates regionStates = am.getRegionStates();
-      assertTrue(TEST_UTIL.assignRegion(hri));
-
-      // Disable the table
-      admin.disableTable(tableName);
-      assertTrue(regionStates.isRegionOffline(hri));
-
-      // You can't assign a disabled region
-      am.assign(hri, true);
-      assertTrue(regionStates.isRegionOffline(hri));
-
-      // You can't unassign a disabled region either
-      am.unassign(hri);
-      assertTrue(regionStates.isRegionOffline(hri));
-    } finally {
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * Test offlined region is assigned by SSH
-   */
-  @Test (timeout=60000)
-  public void testAssignOfflinedRegionBySSH() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    MyMaster master = null;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      // Assign the region
-      master = (MyMaster)cluster.getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      am.assign(hri);
-
-      RegionStates regionStates = am.getRegionStates();
-      ServerName metaServer = regionStates.getRegionServerOfRegion(
-        HRegionInfo.FIRST_META_REGIONINFO);
-      ServerName oldServerName = null;
-      while (true) {
-        assertTrue(am.waitForAssignment(hri));
-        RegionState state = regionStates.getRegionState(hri);
-        oldServerName = state.getServerName();
-        if (!ServerName.isSameHostnameAndPort(oldServerName, metaServer)) {
-          // Mark the hosting server aborted, but don't actually kill it.
-          // It doesn't have meta on it.
-          MyRegionServer.abortedServer = oldServerName;
-          break;
-        }
-        int i = cluster.getServerWithMeta();
-        HRegionServer rs = cluster.getRegionServer(i == 0 ? 1 : 0);
-        oldServerName = rs.getServerName();
-        master.move(hri.getEncodedNameAsBytes(),
-          Bytes.toBytes(oldServerName.getServerName()));
-      }
-
-      // Make sure the region is assigned on the dead server
-      assertTrue(regionStates.isRegionOnline(hri));
-      assertEquals(oldServerName, regionStates.getRegionServerOfRegion(hri));
-
-      // Kill the hosting server, which doesn't have meta on it.
-      cluster.killRegionServer(oldServerName);
-      cluster.waitForRegionServerToStop(oldServerName, -1);
-
-      ServerManager serverManager = master.getServerManager();
-      while (!serverManager.isServerDead(oldServerName)
-          || serverManager.getDeadServers().areDeadServersInProgress()) {
-        Thread.sleep(100);
-      }
-
-      // Let's check if it's assigned after it's out of transition.
-      // no need to assign it manually, SSH should do it
-      am.waitOnRegionToClearRegionsInTransition(hri);
-      assertTrue(am.waitForAssignment(hri));
-
-      ServerName serverName = master.getAssignmentManager().
-        getRegionStates().getRegionServerOfRegion(hri);
-      TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
-    } finally {
-      MyRegionServer.abortedServer = null;
-      TEST_UTIL.deleteTable(tableName);
-      cluster.startRegionServer();
-    }
-  }
-
-  /**
-   * Test disabled region is ignored by SSH
-   */
-  @Test (timeout=60000)
-  public void testAssignDisabledRegionBySSH() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    MyMaster master;
-    try {
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri = new HRegionInfo(
-        desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-
-      // Assign the region
-      master = (MyMaster)cluster.getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      am.assign(hri);
-
-      RegionStates regionStates = am.getRegionStates();
-      ServerName metaServer = regionStates.getRegionServerOfRegion(
-        HRegionInfo.FIRST_META_REGIONINFO);
-      ServerName oldServerName = null;
-      while (true) {
-        assertTrue(am.waitForAssignment(hri));
-        RegionState state = regionStates.getRegionState(hri);
-        oldServerName = state.getServerName();
-        if (!ServerName.isSameHostnameAndPort(oldServerName, metaServer)) {
-          // Mark the hosting server aborted, but don't actually kill it.
-          // It doesn't have meta on it.
-          MyRegionServer.abortedServer = oldServerName;
-          break;
-        }
-        int i = cluster.getServerWithMeta();
-        HRegionServer rs = cluster.getRegionServer(i == 0 ? 1 : 0);
-        oldServerName = rs.getServerName();
-        master.move(hri.getEncodedNameAsBytes(),
-          Bytes.toBytes(oldServerName.getServerName()));
-      }
-
-      // Make sure the region is assigned on the dead server
-      assertTrue(regionStates.isRegionOnline(hri));
-      assertEquals(oldServerName, regionStates.getRegionServerOfRegion(hri));
-
-      // Disable the table now.
-      master.disableTable(hri.getTable(), HConstants.NO_NONCE, HConstants.NO_NONCE);
-
-      // Kill the hosting server, which doesn't have meta on it.
-      cluster.killRegionServer(oldServerName);
-      cluster.waitForRegionServerToStop(oldServerName, -1);
-
-      ServerManager serverManager = master.getServerManager();
-      while (!serverManager.isServerDead(oldServerName)
-          || serverManager.getDeadServers().areDeadServersInProgress()) {
-        Thread.sleep(100);
-      }
-
-      // Wait till no more RIT, the region should be offline.
-      TEST_UTIL.waitUntilNoRegionsInTransition(60000);
-      assertTrue(regionStates.isRegionOffline(hri));
-    } finally {
-      MyRegionServer.abortedServer = null;
-      TEST_UTIL.deleteTable(tableName);
-      cluster.startRegionServer();
-    }
-  }
-
-  /**
-   * Test that region state transition call is idempotent
-   */
-  @Test(timeout = 60000)
-  public void testReportRegionStateTransition() throws Exception {
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try {
-      MyRegionServer.simulateRetry = true;
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(FAMILY));
-      admin.createTable(desc);
-      Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
-      HRegionInfo hri =
-          new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
-      MetaTableAccessor.addRegionToMeta(meta, hri);
-      HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-      AssignmentManager am = master.getAssignmentManager();
-      TEST_UTIL.assignRegion(hri);
-      RegionStates regionStates = am.getRegionStates();
-      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
-      // Assert the the region is actually open on the server
-      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
-      // Closing region should just work fine
-      admin.disableTable(tableName);
-      assertTrue(regionStates.isRegionOffline(hri));
-      List<HRegionInfo> regions = TEST_UTIL.getAdmin().getOnlineRegions(serverName);
-      assertTrue(!regions.contains(hri));
-    } finally {
-      MyRegionServer.simulateRetry = false;
-      TEST_UTIL.deleteTable(tableName);
-    }
-  }
-
-  /**
-   * Test concurrent updates to meta when meta is not on master
-   * @throws Exception
-   */
-  @Test(timeout = 30000)
-  public void testUpdatesRemoteMeta() throws Exception {
-    conf.setInt("hbase.regionstatestore.meta.connection", 3);
-    final RegionStateStore rss =
-        new RegionStateStore(new MyMaster(conf, new ZkCoordinatedStateManager()));
-    rss.start();
-    // Create 10 threads and make each do 10 puts related to region state update
-    Thread[] th = new Thread[10];
-    List<String> nameList = new ArrayList<>();
-    List<TableName> tableNameList = new ArrayList<>();
-    for (int i = 0; i < th.length; i++) {
-      th[i] = new Thread() {
-        @Override
-        public void run() {
-          HRegionInfo[] hri = new HRegionInfo[10];
-          ServerName serverName = ServerName.valueOf("dummyhost", 1000, 1234);
-          for (int i = 0; i < 10; i++) {
-            hri[i] = new HRegionInfo(TableName.valueOf(Thread.currentThread().getName() + "_" + i));
-            RegionState newState = new RegionState(hri[i], RegionState.State.OPEN, serverName);
-            RegionState oldState =
-                new RegionState(hri[i], RegionState.State.PENDING_OPEN, serverName);
-            rss.updateRegionState(1, newState, oldState);
-          }
-        }
-      };
-      th[i].start();
-      nameList.add(th[i].getName());
-    }
-    for (int i = 0; i < th.length; i++) {
-      th[i].join();
-    }
-    // Add all the expected table names in meta to tableNameList
-    for (String name : nameList) {
-      for (int i = 0; i < 10; i++) {
-        tableNameList.add(TableName.valueOf(name + "_" + i));
-      }
-    }
-    List<Result> metaRows = MetaTableAccessor.fullScanRegions(admin.getConnection());
-    int count = 0;
-    // Check all 100 rows are in meta
-    for (Result result : metaRows) {
-      if (tableNameList.contains(HRegionInfo.getTable(result.getRow()))) {
-        count++;
-        if (count == 100) {
-          break;
-        }
-      }
-    }
-    assertTrue(count == 100);
-    rss.stop();
-  }
-
-  static class MyLoadBalancer extends StochasticLoadBalancer {
-    // For this region, if specified, always assign to nowhere
-    static volatile HRegionInfo controledRegion = null;
-
-    static volatile Integer countRegionServers = null;
-    static AtomicInteger counter = new AtomicInteger(0);
-
-    @Override
-    public ServerName randomAssignment(HRegionInfo regionInfo,
-        List<ServerName> servers) {
-      if (regionInfo.equals(controledRegion)) {
-        return null;
-      }
-      return super.randomAssignment(regionInfo, servers);
-    }
-
-    @Override
-    public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
-        List<HRegionInfo> regions, List<ServerName> servers) {
-      if (countRegionServers != null && services != null) {
-        int regionServers = services.getServerManager().countOfRegionServers();
-        if (regionServers < countRegionServers.intValue()) {
-          // Let's wait till more region servers join in.
-          // Before that, fail region assignments.
-          counter.incrementAndGet();
-          return null;
-        }
-      }
-      if (regions.get(0).equals(controledRegion)) {
-        Map<ServerName, List<HRegionInfo>> m = Maps.newHashMap();
-        m.put(LoadBalancer.BOGUS_SERVER_NAME, regions);
-        return m;
-      }
-      return super.roundRobinAssignment(regions, servers);
-    }
-
-    @Override
-    public Map<ServerName, List<HRegionInfo>> retainAssignment(
-        Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
-      for (HRegionInfo hri : regions.keySet()) {
-        if (hri.equals(controledRegion)) {
-          Map<ServerName, List<HRegionInfo>> m = Maps.newHashMap();
-          m.put(LoadBalancer.BOGUS_SERVER_NAME, Lists.newArrayList(regions.keySet()));
-          return m;
-        }
-      }
-      return super.retainAssignment(regions, servers);
-    }
-  }
-
-  public static class MyMaster extends HMaster {
-    AtomicBoolean enabled = new AtomicBoolean(true);
-
-    public MyMaster(Configuration conf, CoordinatedStateManager cp)
-      throws IOException, KeeperException,
-        InterruptedException {
-      super(conf, cp);
-    }
-
-    @Override
-    public boolean isServerCrashProcessingEnabled() {
-      return enabled.get() && super.isServerCrashProcessingEnabled();
-    }
-
-    public void enableSSH(boolean enabled) {
-      this.enabled.set(enabled);
-      if (enabled) {
-        getServerManager().processQueuedDeadServers();
-      }
-    }
-  }
-
-  public static class MyRegionServer extends MiniHBaseClusterRegionServer {
-    static volatile ServerName abortedServer = null;
-    static volatile boolean simulateRetry = false;
-
-    public MyRegionServer(Configuration conf, CoordinatedStateManager cp)
-      throws IOException, KeeperException,
-        InterruptedException {
-      super(conf, cp);
-    }
-
-    @Override
-    public boolean reportRegionStateTransition(TransitionCode code, long openSeqNum,
-        HRegionInfo... hris) {
-      if (simulateRetry) {
-        // Simulate retry by calling the method twice
-        super.reportRegionStateTransition(code, openSeqNum, hris);
-        return super.reportRegionStateTransition(code, openSeqNum, hris);
-      }
-      return super.reportRegionStateTransition(code, openSeqNum, hris);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return getServerName().equals(abortedServer) || super.isAborted();
-    }
-  }
-
-  public static class MyRegionObserver implements RegionObserver {
-    // If enabled, fail all preClose calls
-    static AtomicBoolean preCloseEnabled = new AtomicBoolean(false);
-
-    // If enabled, stall postClose calls
-    static AtomicBoolean postCloseEnabled = new AtomicBoolean(false);
-
-    // If enabled, stall postOpen calls
-    static AtomicBoolean postOpenEnabled = new AtomicBoolean(false);
-
-    // A flag to track if postOpen is called
-    static volatile boolean postOpenCalled = false;
-
-    @Override
-    public void preClose(ObserverContext<RegionCoprocessorEnvironment> c,
-        boolean abortRequested) throws IOException {
-      if (preCloseEnabled.get()) throw new IOException("fail preClose from coprocessor");
-    }
-
-    @Override
-    public void postClose(ObserverContext<RegionCoprocessorEnvironment> c,
-        boolean abortRequested) {
-      stallOnFlag(postCloseEnabled);
-    }
-
-    @Override
-    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
-      postOpenCalled = true;
-      stallOnFlag(postOpenEnabled);
-    }
-
-    private void stallOnFlag(final AtomicBoolean flag) {
-      try {
-        // If enabled, stall
-        while (flag.get()) {
-          Thread.sleep(1000);
-        }
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index cc73d9d..b6a0035 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
 import org.apache.hadoop.hbase.io.Reference;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index b78bfd1..fcd2cf9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index eb4ce99..d6210b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
index 74f2c91..c020064 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterBalanceThrottling.java
@@ -120,8 +120,9 @@ public class TestMasterBalanceThrottling {
       @Override
       public void run() {
         while (!stop.get()) {
-          maxCount.set(Math.max(maxCount.get(), master.getAssignmentManager().getRegionStates()
-              .getRegionsInTransitionCount()));
+          maxCount.set(Math.max(maxCount.get(),
+              master.getAssignmentManager().getRegionStates()
+              .getRegionsInTransition().size()));
           try {
             Thread.sleep(10);
           } catch (InterruptedException e) {
@@ -136,7 +137,7 @@ public class TestMasterBalanceThrottling {
   }
 
   private void unbalance(HMaster master, TableName tableName) throws Exception {
-    while (master.getAssignmentManager().getRegionStates().getRegionsInTransitionCount() > 0) {
+    while (master.getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0) {
       Thread.sleep(100);
     }
     HRegionServer biasedServer = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
@@ -144,7 +145,7 @@ public class TestMasterBalanceThrottling {
       master.move(regionInfo.getEncodedNameAsBytes(),
         Bytes.toBytes(biasedServer.getServerName().getServerName()));
     }
-    while (master.getAssignmentManager().getRegionStates().getRegionsInTransitionCount() > 0) {
+    while (master.getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0) {
       Thread.sleep(100);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index f57d6b9..986e8d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -252,21 +254,20 @@ public class TestMasterFailover {
     // Put the online region in pending_close. It is actually already opened.
     // This is to simulate that the region close RPC is not sent out before failover
     RegionState oldState = regionStates.getRegionState(hriOnline);
-    RegionState newState = new RegionState(
-      hriOnline, State.PENDING_CLOSE, oldState.getServerName());
+    RegionState newState = new RegionState(hriOnline, State.CLOSING, oldState.getServerName());
     stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
     // Put the offline region in pending_open. It is actually not opened yet.
     // This is to simulate that the region open RPC is not sent out before failover
     oldState = new RegionState(hriOffline, State.OFFLINE);
-    newState = new RegionState(hriOffline, State.PENDING_OPEN, newState.getServerName());
+    newState = new RegionState(hriOffline, State.OPENING, newState.getServerName());
     stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
     HRegionInfo failedClose = new HRegionInfo(offlineTable.getTableName(), null, null);
     createRegion(failedClose, rootdir, conf, offlineTable);
     MetaTableAccessor.addRegionToMeta(master.getConnection(), failedClose);
 
-    oldState = new RegionState(failedClose, State.PENDING_CLOSE);
+    oldState = new RegionState(failedClose, State.CLOSING);
     newState = new RegionState(failedClose, State.FAILED_CLOSE, newState.getServerName());
     stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
@@ -276,7 +277,7 @@ public class TestMasterFailover {
 
     // Simulate a region transitioning to failed open when the region server reports the
     // transition as FAILED_OPEN
-    oldState = new RegionState(failedOpen, State.PENDING_OPEN);
+    oldState = new RegionState(failedOpen, State.OPENING);
     newState = new RegionState(failedOpen, State.FAILED_OPEN, newState.getServerName());
     stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
@@ -378,12 +379,12 @@ public class TestMasterFailover {
     assertEquals("hbase:meta should be onlined on RS",
       metaState.getState(), State.OPEN);
 
-    // Update meta state as PENDING_OPEN, then kill master
+    // Update meta state as OPENING, then kill master
     // that simulates, that RS successfully deployed, but
     // RPC was lost right before failure.
     // region server should expire (how it can be verified?)
     MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
-      rs.getServerName(), State.PENDING_OPEN);
+      rs.getServerName(), State.OPENING);
     Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     rs.removeFromOnlineRegions(meta, null);
     ((HRegion)meta).close();
@@ -410,12 +411,12 @@ public class TestMasterFailover {
     assertEquals("hbase:meta should be onlined on RS",
       metaState.getState(), State.OPEN);
 
-    // Update meta state as PENDING_CLOSE, then kill master
+    // Update meta state as CLOSING, then kill master
     // that simulates, that RS successfully deployed, but
     // RPC was lost right before failure.
     // region server should expire (how it can be verified?)
     MetaTableLocator.setMetaLocation(activeMaster.getZooKeeper(),
-      rs.getServerName(), State.PENDING_CLOSE);
+      rs.getServerName(), State.CLOSING);
 
     log("Aborting master");
     activeMaster.abort("test-kill");

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
index b59e6ff..478e4d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterStatusServlet.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -164,7 +166,8 @@ public class TestMasterStatusServlet {
     RegionStates rs = Mockito.mock(RegionStates.class);
 
     // Add 100 regions as in-transition
-    TreeSet<RegionState> regionsInTransition = new TreeSet<>(RegionStates.REGION_STATE_COMPARATOR);
+    TreeSet<RegionState> regionsInTransition = new TreeSet<>(
+      RegionStates.REGION_STATE_STAMP_COMPARATOR);
     for (byte i = 0; i < 100; i++) {
       HRegionInfo hri = new HRegionInfo(FAKE_TABLE.getTableName(),
           new byte[]{i}, new byte[]{(byte) (i+1)});

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
index a845a73..68160df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java
index daf6d43..fe5883b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionState.java
@@ -35,14 +35,19 @@ public class TestRegionState {
   public TestName name = new TestName();
 
   @Test
-  public void test() {
-    RegionState state1 = new RegionState(
-            new HRegionInfo(TableName.valueOf(name.getMethodName())), RegionState.State.OPENING);
+  public void testSerializeDeserialize() {
+    final TableName tableName = TableName.valueOf("testtb");
+    for (RegionState.State state: RegionState.State.values()) {
+      testSerializeDeserialize(tableName, state);
+    }
+  }
+
+  private void testSerializeDeserialize(final TableName tableName, final RegionState.State state) {
+    RegionState state1 = new RegionState(new HRegionInfo(tableName), state);
     ClusterStatusProtos.RegionState protobuf1 = state1.convert();
     RegionState state2 = RegionState.convert(protobuf1);
     ClusterStatusProtos.RegionState protobuf2 = state1.convert();
-
-    assertEquals(state1, state2);
-    assertEquals(protobuf1, protobuf2);
+    assertEquals("RegionState does not match " + state, state1, state2);
+    assertEquals("Protobuf does not match " + state, protobuf1, protobuf2);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
deleted file mode 100644
index 17004ec..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
+++ /dev/null
@@ -1,144 +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;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CyclicBarrier;
-
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import static org.junit.Assert.assertTrue;
-import static junit.framework.Assert.assertFalse;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-@Category({MasterTests.class, SmallTests.class})
-public class TestRegionStates {
-  @Test (timeout=10000)
-  public void testCanMakeProgressThoughMetaIsDown()
-  throws IOException, InterruptedException, BrokenBarrierException {
-    MasterServices server = mock(MasterServices.class);
-    when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
-    Connection connection = mock(ClusterConnection.class);
-    // Set up a table that gets 'stuck' when we try to fetch a row from the meta table.
-    // It is stuck on a CyclicBarrier latch. We use CyclicBarrier because it will tell us when
-    // thread is waiting on latch.
-    Table metaTable = Mockito.mock(Table.class);
-    final CyclicBarrier latch = new CyclicBarrier(2);
-    when(metaTable.get((Get)Mockito.any())).thenAnswer(new Answer<Result>() {
-      @Override
-      public Result answer(InvocationOnMock invocation) throws Throwable {
-        latch.await();
-        throw new java.net.ConnectException("Connection refused");
-      }
-    });
-    when(connection.getTable(TableName.META_TABLE_NAME)).thenReturn(metaTable);
-    when(server.getConnection()).thenReturn((ClusterConnection)connection);
-    Configuration configuration = mock(Configuration.class);
-    when(server.getConfiguration()).thenReturn(configuration);
-    TableStateManager tsm = mock(TableStateManager.class);
-    ServerManager sm = mock(ServerManager.class);
-    when(sm.isServerOnline(isA(ServerName.class))).thenReturn(true);
-
-    RegionStateStore rss = mock(RegionStateStore.class);
-    final RegionStates regionStates = new RegionStates(server, tsm, sm, rss);
-    final ServerName sn = mockServer("one", 1);
-    regionStates.updateRegionState(HRegionInfo.FIRST_META_REGIONINFO, State.SPLITTING_NEW, sn);
-    Thread backgroundThread = new Thread("Get stuck setting server offline") {
-      @Override
-      public void run() {
-        regionStates.serverOffline(sn);
-      }
-    };
-    assertTrue(latch.getNumberWaiting() == 0);
-    backgroundThread.start();
-    while (latch.getNumberWaiting() == 0);
-    // Verify I can do stuff with synchronized RegionStates methods, that I am not locked out.
-    // Below is a call that is synchronized.  Can I do it and not block?
-    regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO);
-    // Done. Trip the barrier on the background thread.
-    latch.await();
-  }
-
-  @Test
-  public void testWeDontReturnDrainingServersForOurBalancePlans() throws Exception {
-    MasterServices server = mock(MasterServices.class);
-    when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
-    Configuration configuration = mock(Configuration.class);
-    when(server.getConfiguration()).thenReturn(configuration);
-    TableStateManager tsm = mock(TableStateManager.class);
-    ServerManager sm = mock(ServerManager.class);
-    when(sm.isServerOnline(isA(ServerName.class))).thenReturn(true);
-
-    RegionStateStore rss = mock(RegionStateStore.class);
-    RegionStates regionStates = new RegionStates(server, tsm, sm, rss);
-
-    ServerName one = mockServer("one", 1);
-    ServerName two = mockServer("two", 1);
-    ServerName three = mockServer("three", 1);
-
-    when(sm.getDrainingServersList()).thenReturn(Arrays.asList(three));
-
-    regionStates.regionOnline(createFakeRegion(), one);
-    regionStates.regionOnline(createFakeRegion(), two);
-    regionStates.regionOnline(createFakeRegion(), three);
-
-
-    Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
-        regionStates.getAssignmentsByTable();
-    for (Map<ServerName, List<HRegionInfo>> map : result.values()) {
-      assertFalse(map.keySet().contains(three));
-    }
-  }
-
-  private HRegionInfo createFakeRegion() {
-    HRegionInfo info = mock(HRegionInfo.class);
-    when(info.getEncodedName()).thenReturn(UUID.randomUUID().toString());
-    return info;
-  }
-
-  private ServerName mockServer(String fakeHost, int fakePort) {
-    ServerName serverName = mock(ServerName.class);
-    when(serverName.getHostname()).thenReturn(fakeHost);
-    when(serverName.getPort()).thenReturn(fakePort);
-    return serverName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
index 7c41c0f..3e3af27 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
new file mode 100644
index 0000000..07b989b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
@@ -0,0 +1,125 @@
+/**
+ * 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.assignment;
+
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.util.Threads;
+
+import static org.junit.Assert.assertEquals;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class AssignmentTestingUtil {
+  private static final Log LOG = LogFactory.getLog(AssignmentTestingUtil.class);
+
+  private AssignmentTestingUtil() {}
+
+  public static void waitForRegionToBeInTransition(final HBaseTestingUtility util,
+      final HRegionInfo hri) throws Exception {
+    while (!getMaster(util).getAssignmentManager().getRegionStates().isRegionInTransition(hri)) {
+      Threads.sleep(10);
+    }
+  }
+
+  public static void waitForRsToBeDead(final HBaseTestingUtility util,
+      final ServerName serverName) throws Exception {
+    util.waitFor(60000, new ExplainingPredicate<Exception>() {
+      @Override
+      public boolean evaluate() {
+        return getMaster(util).getServerManager().isServerDead(serverName);
+      }
+
+      @Override
+      public String explainFailure() {
+        return "Server " + serverName + " is not dead";
+      }
+    });
+  }
+
+  public static void stopRs(final HBaseTestingUtility util, final ServerName serverName)
+      throws Exception {
+    LOG.info("STOP REGION SERVER " + serverName);
+    util.getMiniHBaseCluster().stopRegionServer(serverName);
+    waitForRsToBeDead(util, serverName);
+  }
+
+  public static void killRs(final HBaseTestingUtility util, final ServerName serverName)
+      throws Exception {
+    LOG.info("KILL REGION SERVER " + serverName);
+    util.getMiniHBaseCluster().killRegionServer(serverName);
+    waitForRsToBeDead(util, serverName);
+  }
+
+  public static void crashRs(final HBaseTestingUtility util, final ServerName serverName,
+      final boolean kill) throws Exception {
+    if (kill) {
+      killRs(util, serverName);
+    } else {
+      stopRs(util, serverName);
+    }
+  }
+
+  public static ServerName crashRsWithRegion(final HBaseTestingUtility util,
+      final HRegionInfo hri, final boolean kill) throws Exception {
+    ServerName serverName = getServerHoldingRegion(util, hri);
+    crashRs(util, serverName, kill);
+    return serverName;
+  }
+
+  public static ServerName getServerHoldingRegion(final HBaseTestingUtility util,
+      final HRegionInfo hri) throws Exception {
+    ServerName serverName = util.getMiniHBaseCluster().getServerHoldingRegion(
+      hri.getTable(), hri.getRegionName());
+    ServerName amServerName = getMaster(util).getAssignmentManager().getRegionStates()
+      .getRegionServerOfRegion(hri);
+
+    // Make sure AM and MiniCluster agrees on the Server holding the region
+    // and that the server is online.
+    assertEquals(amServerName, serverName);
+    assertEquals(true, getMaster(util).getServerManager().isServerOnline(serverName));
+    return serverName;
+  }
+
+  public static boolean isServerHoldingMeta(final HBaseTestingUtility util,
+      final ServerName serverName) throws Exception {
+    for (HRegionInfo hri: getMetaRegions(util)) {
+      if (serverName.equals(getServerHoldingRegion(util, hri))) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public static Set<HRegionInfo> getMetaRegions(final HBaseTestingUtility util) {
+    return getMaster(util).getAssignmentManager().getMetaRegionSet();
+  }
+
+  private static HMaster getMaster(final HBaseTestingUtility util) {
+    return util.getMiniHBaseCluster().getMaster();
+  }
+}
\ No newline at end of file


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
new file mode 100644
index 0000000..022cdec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -0,0 +1,1660 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.PleaseHoldException;
+import org.apache.hadoop.hbase.RegionException;
+import org.apache.hadoop.hbase.RegionStateListener;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
+import org.apache.hadoop.hbase.favored.FavoredNodeLoadBalancer;
+import org.apache.hadoop.hbase.master.AssignmentListener;
+import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.ServerListener;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerState;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.ServerStateNode;
+// TODO: why are they here?
+import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.quotas.QuotaExceededException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.Threads;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * The AssignmentManager is the coordinator for region assign/unassign operations.
+ * <ul>
+ * <li>In-memory states of regions and servers are stored in {@link RegionStates}.</li>
+ * <li>hbase:meta state updates are handled by {@link RegionStateStore}.</li>
+ * </ul>
+ * Regions are created by CreateTable, Split, Merge.
+ * Regions are deleted by DeleteTable, Split, Merge.
+ * Assigns are triggered by CreateTable, EnableTable, Split, Merge, ServerCrash.
+ * Unassigns are triggered by DisableTable, Split, Merge
+ */
+@InterfaceAudience.Private
+public class AssignmentManager implements ServerListener {
+  private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
+
+  // TODO: AMv2
+  //  - handle region migration
+  //  - handle meta assignment first <= IS THIS DONE?
+  //  - handle sys table assignment first (e.g. acl, namespace)
+  //  - handle table priorities <= IS THIS DONE?
+  //  - If ServerBusyException trying to update hbase:meta, we abort the Master
+  //   See updateRegionLocation in RegionStateStore.
+
+  public static final String BOOTSTRAP_THREAD_POOL_SIZE_CONF_KEY =
+      "hbase.assignment.bootstrap.thread.pool.size";
+
+  public static final String ASSIGN_DISPATCH_WAIT_MSEC_CONF_KEY =
+      "hbase.assignment.dispatch.wait.msec";
+  private static final int DEFAULT_ASSIGN_DISPATCH_WAIT_MSEC = 150;
+
+  public static final String ASSIGN_DISPATCH_WAITQ_MAX_CONF_KEY =
+      "hbase.assignment.dispatch.wait.queue.max.size";
+  private static final int DEFAULT_ASSIGN_DISPATCH_WAITQ_MAX = 100;
+
+  public static final String RIT_CHORE_INTERVAL_MSEC_CONF_KEY =
+      "hbase.assignment.rit.chore.interval.msec";
+  private static final int DEFAULT_RIT_CHORE_INTERVAL_MSEC = 5 * 1000;
+
+  public static final String ASSIGN_MAX_ATTEMPTS =
+      "hbase.assignment.maximum.attempts";
+  private static final int DEFAULT_ASSIGN_MAX_ATTEMPTS = 10;
+
+  /** Region in Transition metrics threshold time */
+  public static final String METRICS_RIT_STUCK_WARNING_THRESHOLD =
+      "hbase.metrics.rit.stuck.warning.threshold";
+  private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
+
+  private final ProcedureEvent<?> metaInitializedEvent = new ProcedureEvent<>("meta initialized");
+  private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
+
+  /**
+   * Indicator that AssignmentManager has recovered the region states so
+   * that ServerCrashProcedure can be fully enabled and re-assign regions
+   * of dead servers. So that when re-assignment happens, AssignmentManager
+   * has proper region states.
+   */
+  private final ProcedureEvent<?> failoverCleanupDone = new ProcedureEvent<>("failover cleanup");
+
+  /** Listeners that are called on assignment events. */
+  private final CopyOnWriteArrayList<AssignmentListener> listeners =
+      new CopyOnWriteArrayList<AssignmentListener>();
+
+  // TODO: why is this different from the listeners (carried over from the old AM)
+  private RegionStateListener regionStateListener;
+
+  private final MetricsAssignmentManager metrics;
+  private final RegionInTransitionChore ritChore;
+  private final MasterServices master;
+
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final RegionStates regionStateMap = new RegionStates();
+  private final RegionStateStore regionStateStore;
+
+  private final boolean shouldAssignRegionsWithFavoredNodes;
+  private final int assignDispatchWaitQueueMaxSize;
+  private final int assignDispatchWaitMillis;
+  private final int assignMaxAttempts;
+
+  private Thread assignThread;
+
+  public AssignmentManager(final MasterServices master) {
+    this(master, new RegionStateStore(master));
+  }
+
+  public AssignmentManager(final MasterServices master, final RegionStateStore stateStore) {
+    this.master = master;
+    this.regionStateStore = stateStore;
+    this.metrics = new MetricsAssignmentManager();
+
+    final Configuration conf = master.getConfiguration();
+
+    // Only read favored nodes if using the favored nodes load balancer.
+    this.shouldAssignRegionsWithFavoredNodes = FavoredNodeLoadBalancer.class.isAssignableFrom(
+        conf.getClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class));
+
+    this.assignDispatchWaitMillis = conf.getInt(ASSIGN_DISPATCH_WAIT_MSEC_CONF_KEY,
+        DEFAULT_ASSIGN_DISPATCH_WAIT_MSEC);
+    this.assignDispatchWaitQueueMaxSize = conf.getInt(ASSIGN_DISPATCH_WAITQ_MAX_CONF_KEY,
+        DEFAULT_ASSIGN_DISPATCH_WAITQ_MAX);
+
+    this.assignMaxAttempts = Math.max(1, conf.getInt(ASSIGN_MAX_ATTEMPTS,
+        DEFAULT_ASSIGN_MAX_ATTEMPTS));
+
+    int ritChoreInterval = conf.getInt(RIT_CHORE_INTERVAL_MSEC_CONF_KEY,
+        DEFAULT_RIT_CHORE_INTERVAL_MSEC);
+    this.ritChore = new RegionInTransitionChore(ritChoreInterval);
+  }
+
+  public void start() throws IOException {
+    if (!running.compareAndSet(false, true)) {
+      return;
+    }
+
+    LOG.info("Starting assignment manager");
+
+    // Register Server Listener
+    master.getServerManager().registerListener(this);
+
+    // Start the RegionStateStore
+    regionStateStore.start();
+
+    // Start the Assignment Thread
+    startAssignmentThread();
+  }
+
+  public void stop() {
+    if (!running.compareAndSet(true, false)) {
+      return;
+    }
+
+    LOG.info("Stopping assignment manager");
+
+    // The AM is started before the procedure executor,
+    // but the actual work will be loaded/submitted only once we have the executor
+    final boolean hasProcExecutor = master.getMasterProcedureExecutor() != null;
+
+    // Remove the RIT chore
+    if (hasProcExecutor) {
+      master.getMasterProcedureExecutor().removeChore(this.ritChore);
+    }
+
+    // Stop the Assignment Thread
+    stopAssignmentThread();
+
+    // Stop the RegionStateStore
+    regionStateMap.clear();
+    regionStateStore.stop();
+
+    // Unregister Server Listener
+    master.getServerManager().unregisterListener(this);
+
+    // Update meta events (for testing)
+    if (hasProcExecutor) {
+      getProcedureScheduler().suspendEvent(metaLoadEvent);
+      setFailoverCleanupDone(false);
+      for (HRegionInfo hri: getMetaRegionSet()) {
+        setMetaInitialized(hri, false);
+      }
+    }
+  }
+
+  public boolean isRunning() {
+    return running.get();
+  }
+
+  public Configuration getConfiguration() {
+    return master.getConfiguration();
+  }
+
+  public MetricsAssignmentManager getAssignmentManagerMetrics() {
+    return metrics;
+  }
+
+  private LoadBalancer getBalancer() {
+    return master.getLoadBalancer();
+  }
+
+  private MasterProcedureEnv getProcedureEnvironment() {
+    return master.getMasterProcedureExecutor().getEnvironment();
+  }
+
+  private MasterProcedureScheduler getProcedureScheduler() {
+    return getProcedureEnvironment().getProcedureScheduler();
+  }
+
+  protected int getAssignMaxAttempts() {
+    return assignMaxAttempts;
+  }
+
+  /**
+   * Add the listener to the notification list.
+   * @param listener The AssignmentListener to register
+   */
+  public void registerListener(final AssignmentListener listener) {
+    this.listeners.add(listener);
+  }
+
+  /**
+   * Remove the listener from the notification list.
+   * @param listener The AssignmentListener to unregister
+   */
+  public boolean unregisterListener(final AssignmentListener listener) {
+    return this.listeners.remove(listener);
+  }
+
+  public void setRegionStateListener(final RegionStateListener listener) {
+    this.regionStateListener = listener;
+  }
+
+  // TODO: Why public?
+  public RegionStates getRegionStates() {
+    return regionStateMap;
+  }
+
+  // TODO: Why public?
+  public RegionStateStore getRegionStateStore() {
+    return regionStateStore;
+  }
+
+  public List<ServerName> getFavoredNodes(final HRegionInfo regionInfo) {
+    if (shouldAssignRegionsWithFavoredNodes) {
+      return ((FavoredNodeLoadBalancer)getBalancer()).getFavoredNodes(regionInfo);
+    }
+    return ServerName.EMPTY_SERVER_LIST;
+  }
+
+  // ============================================================================================
+  //  Table State Manager helpers
+  // ============================================================================================
+  // TODO: Why public?
+  public TableStateManager getTableStateManager() {
+    return master.getTableStateManager();
+  }
+
+  public boolean isTableEnabled(final TableName tableName) {
+    return getTableStateManager().isTableState(tableName, TableState.State.ENABLED);
+  }
+
+  public boolean isTableDisabled(final TableName tableName) {
+    return getTableStateManager().isTableState(tableName,
+      TableState.State.DISABLED, TableState.State.DISABLING);
+  }
+
+  // ============================================================================================
+  //  META Helpers
+  // ============================================================================================
+  private boolean isMetaRegion(final HRegionInfo regionInfo) {
+    return regionInfo.isMetaRegion();
+  }
+
+  public boolean isMetaRegion(final byte[] regionName) {
+    return getMetaRegionFromName(regionName) != null;
+  }
+
+  public HRegionInfo getMetaRegionFromName(final byte[] regionName) {
+    for (HRegionInfo hri: getMetaRegionSet()) {
+      if (Bytes.equals(hri.getRegionName(), regionName)) {
+        return hri;
+      }
+    }
+    return null;
+  }
+
+  public boolean isCarryingMeta(final ServerName serverName) {
+    for (HRegionInfo hri: getMetaRegionSet()) {
+      if (isCarryingMeta(serverName, hri)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public boolean isCarryingMeta(final ServerName serverName, final HRegionInfo regionInfo) {
+    return isCarryingRegion(serverName, regionInfo);
+  }
+
+  public boolean isCarryingMetaReplica(final ServerName serverName, final HRegionInfo regionInfo) {
+    return isCarryingRegion(serverName, regionInfo);
+  }
+
+  private boolean isCarryingRegion(final ServerName serverName, final HRegionInfo regionInfo) {
+    // TODO: check for state?
+    final RegionStateNode node = regionStateMap.getRegionNode(regionInfo);
+    return(node != null && serverName.equals(node.getRegionLocation()));
+  }
+
+  private HRegionInfo getMetaForRegion(final HRegionInfo regionInfo) {
+    //if (regionInfo.isMetaRegion()) return regionInfo;
+    // TODO: handle multiple meta. if the region provided is not meta lookup
+    // which meta the region belongs to.
+    return HRegionInfo.FIRST_META_REGIONINFO;
+  }
+
+  // TODO: handle multiple meta.
+  private static final Set<HRegionInfo> META_REGION_SET =
+      Collections.singleton(HRegionInfo.FIRST_META_REGIONINFO);
+  public Set<HRegionInfo> getMetaRegionSet() {
+    return META_REGION_SET;
+  }
+
+  // ============================================================================================
+  //  META Event(s) helpers
+  // ============================================================================================
+  public boolean isMetaInitialized() {
+    return metaInitializedEvent.isReady();
+  }
+
+  public boolean isMetaRegionInTransition() {
+    return !isMetaInitialized();
+  }
+
+  public boolean waitMetaInitialized(final Procedure proc) {
+    // TODO: handle multiple meta. should this wait on all meta?
+    // this is used by the ServerCrashProcedure...
+    return waitMetaInitialized(proc, HRegionInfo.FIRST_META_REGIONINFO);
+  }
+
+  public boolean waitMetaInitialized(final Procedure proc, final HRegionInfo regionInfo) {
+    return getProcedureScheduler().waitEvent(
+      getMetaInitializedEvent(getMetaForRegion(regionInfo)), proc);
+  }
+
+  private void setMetaInitialized(final HRegionInfo metaRegionInfo, final boolean isInitialized) {
+    assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
+    final ProcedureEvent metaInitEvent = getMetaInitializedEvent(metaRegionInfo);
+    if (isInitialized) {
+      getProcedureScheduler().wakeEvent(metaInitEvent);
+    } else {
+      getProcedureScheduler().suspendEvent(metaInitEvent);
+    }
+  }
+
+  private ProcedureEvent getMetaInitializedEvent(final HRegionInfo metaRegionInfo) {
+    assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
+    // TODO: handle multiple meta.
+    return metaInitializedEvent;
+  }
+
+  public boolean waitMetaLoaded(final Procedure proc) {
+    return getProcedureScheduler().waitEvent(metaLoadEvent, proc);
+  }
+
+  protected void wakeMetaLoadedEvent() {
+    getProcedureScheduler().wakeEvent(metaLoadEvent);
+    assert isMetaLoaded() : "expected meta to be loaded";
+  }
+
+  public boolean isMetaLoaded() {
+    return metaLoadEvent.isReady();
+  }
+
+  // ============================================================================================
+  //  TODO: Sync helpers
+  // ============================================================================================
+  public void assignMeta(final HRegionInfo metaRegionInfo) throws IOException {
+    assignMeta(metaRegionInfo, null);
+  }
+
+  public void assignMeta(final HRegionInfo metaRegionInfo, final ServerName serverName)
+      throws IOException {
+    assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
+    AssignProcedure proc;
+    if (serverName != null) {
+      LOG.debug("Try assigning Meta " + metaRegionInfo + " to " + serverName);
+      proc = createAssignProcedure(metaRegionInfo, serverName);
+    } else {
+      LOG.debug("Assigning " + metaRegionInfo.getRegionNameAsString());
+      proc = createAssignProcedure(metaRegionInfo, false);
+    }
+    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
+  }
+
+  public void assign(final HRegionInfo regionInfo) throws IOException {
+    assign(regionInfo, true);
+  }
+
+  public void assign(final HRegionInfo regionInfo, final boolean forceNewPlan) throws IOException {
+    AssignProcedure proc = createAssignProcedure(regionInfo, forceNewPlan);
+    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
+  }
+
+  public void unassign(final HRegionInfo regionInfo) {
+    // TODO: rename this in reassign
+  }
+
+  @VisibleForTesting
+  public void reopen(final HRegionInfo region) {
+    // TODO: used by TestScannersFromClientSide.java???
+  }
+
+  public Future<byte[]> moveAsync(final RegionPlan regionPlan) {
+    MoveRegionProcedure proc = createMoveRegionProcedure(regionPlan);
+    return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
+  }
+
+  @VisibleForTesting
+  public boolean waitForAssignment(final HRegionInfo regionInfo) throws IOException {
+    return waitForAssignment(regionInfo, Long.MAX_VALUE);
+  }
+
+  @VisibleForTesting
+  // TODO: Remove this?
+  public boolean waitForAssignment(final HRegionInfo regionInfo, final long timeout)
+  throws IOException {
+    RegionStateNode node = null;
+    // This method can be called before the regionInfo has made it into the regionStateMap
+    // so wait around here a while.
+    long startTime = System.currentTimeMillis();
+    long endTime = startTime + 10000; // Something badly wrong if takes ten seconds to register a region.
+    while ((node = regionStateMap.getRegionNode(regionInfo)) == null && isRunning() &&
+        System.currentTimeMillis() < endTime) {
+      // Presume it not yet added but will be added soon. Let it spew a lot so we can tell if
+      // we are waiting here alot.
+      LOG.debug("Waiting on " + regionInfo + " to be added to regionStateMap");
+      Threads.sleep(10);
+    }
+    if (node == null) {
+      if (!isRunning()) return false;
+      throw new RegionException(regionInfo.getRegionNameAsString() + " never registered with Assigment.");
+    }
+
+    RegionTransitionProcedure proc = node.getProcedure();
+    if (proc == null) {
+      // return false;
+      throw new HBaseIOException("This should not happen!!!! FIX!!!");
+    }
+
+    ProcedureSyncWait.waitForProcedureToCompleteIOE(
+      master.getMasterProcedureExecutor(), proc.getProcId(), timeout);
+    return true;
+  }
+
+  // ============================================================================================
+  //  RegionTransition procedures helpers
+  // ============================================================================================
+  public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo) {
+    return createAssignProcedures(regionInfo, false);
+  }
+
+  public AssignProcedure[] createAssignProcedures(final Collection<HRegionInfo> regionInfo,
+      final boolean forceNewPlan) {
+    final AssignProcedure[] procs = new AssignProcedure[regionInfo.size()];
+    int index = 0;
+    for (HRegionInfo hri: regionInfo) {
+      procs[index++] = createAssignProcedure(hri, forceNewPlan);
+    }
+    return procs;
+  }
+
+  public UnassignProcedure[] createUnassignProcedures(final Collection<HRegionInfo> regionInfo) {
+    final UnassignProcedure[] procs = new UnassignProcedure[regionInfo.size()];
+    int index = 0;
+    for (HRegionInfo hri: regionInfo) {
+      procs[index++] = createUnassignProcedure(hri, null, false);
+    }
+    return procs;
+  }
+
+  public MoveRegionProcedure[] createReopenProcedures(final Collection<HRegionInfo> regionInfo) {
+    final MoveRegionProcedure[] procs = new MoveRegionProcedure[regionInfo.size()];
+    int index = 0;
+    for (HRegionInfo hri: regionInfo) {
+      final ServerName serverName = regionStateMap.getRegionServerOfRegion(hri);
+      final RegionPlan plan = new RegionPlan(hri, serverName, serverName);
+      procs[index++] = createMoveRegionProcedure(plan);
+    }
+    return procs;
+  }
+
+  /**
+   * Called by things like EnableTableProcedure to get a list of AssignProcedure
+   * to assign the regions of the table.
+   */
+  public AssignProcedure[] createAssignProcedures(final TableName tableName) {
+    return createAssignProcedures(regionStateMap.getRegionsOfTable(tableName));
+  }
+
+  /**
+   * Called by things like DisableTableProcedure to get a list of UnassignProcedure
+   * to unassign the regions of the table.
+   */
+  public UnassignProcedure[] createUnassignProcedures(final TableName tableName) {
+    return createUnassignProcedures(regionStateMap.getRegionsOfTable(tableName));
+  }
+
+  /**
+   * Called by things like ModifyColumnFamilyProcedure to get a list of MoveRegionProcedure
+   * to reopen the regions of the table.
+   */
+  public MoveRegionProcedure[] createReopenProcedures(final TableName tableName) {
+    return createReopenProcedures(regionStateMap.getRegionsOfTable(tableName));
+  }
+
+  public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
+      final boolean forceNewPlan) {
+    AssignProcedure proc = new AssignProcedure(regionInfo, forceNewPlan);
+    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
+    return proc;
+  }
+
+  public AssignProcedure createAssignProcedure(final HRegionInfo regionInfo,
+      final ServerName targetServer) {
+    AssignProcedure proc = new AssignProcedure(regionInfo, targetServer);
+    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
+    return proc;
+  }
+
+  public UnassignProcedure createUnassignProcedure(final HRegionInfo regionInfo,
+      final ServerName destinationServer, final boolean force) {
+    UnassignProcedure proc = new UnassignProcedure(regionInfo, destinationServer, force);
+    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
+    return proc;
+  }
+
+  public MoveRegionProcedure createMoveRegionProcedure(final RegionPlan plan) {
+    MoveRegionProcedure proc = new MoveRegionProcedure(plan);
+    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
+    return proc;
+  }
+
+
+  public SplitTableRegionProcedure createSplitProcedure(final HRegionInfo regionToSplit,
+      final byte[] splitKey) throws IOException {
+    return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
+  }
+
+  public MergeTableRegionsProcedure createMergeProcedure(final HRegionInfo regionToMergeA,
+      final HRegionInfo regionToMergeB) throws IOException {
+    return new MergeTableRegionsProcedure(getProcedureEnvironment(), regionToMergeA,regionToMergeB);
+  }
+
+  /**
+   * Delete the region states. This is called by "DeleteTable"
+   */
+  public void deleteTable(final TableName tableName) throws IOException {
+    final ArrayList<HRegionInfo> regions = regionStateMap.getTableRegionsInfo(tableName);
+    regionStateStore.deleteRegions(regions);
+    for (int i = 0; i < regions.size(); ++i) {
+      final HRegionInfo regionInfo = regions.get(i);
+      // we expect the region to be offline
+      regionStateMap.removeFromOfflineRegions(regionInfo);
+      regionStateMap.deleteRegion(regionInfo);
+    }
+  }
+
+  // ============================================================================================
+  //  RS Region Transition Report helpers
+  // ============================================================================================
+  // TODO: Move this code in MasterRpcServices and call on specific event?
+  public ReportRegionStateTransitionResponse reportRegionStateTransition(
+      final ReportRegionStateTransitionRequest req)
+  throws PleaseHoldException {
+    final ReportRegionStateTransitionResponse.Builder builder =
+        ReportRegionStateTransitionResponse.newBuilder();
+    final ServerName serverName = ProtobufUtil.toServerName(req.getServer());
+    try {
+      for (RegionStateTransition transition: req.getTransitionList()) {
+        switch (transition.getTransitionCode()) {
+          case OPENED:
+          case FAILED_OPEN:
+          case CLOSED:
+            assert transition.getRegionInfoCount() == 1 : transition;
+            final HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
+            updateRegionTransition(serverName, transition.getTransitionCode(), hri,
+                transition.hasOpenSeqNum() ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM);
+            break;
+          case READY_TO_SPLIT:
+          case SPLIT_PONR:
+          case SPLIT:
+          case SPLIT_REVERTED:
+            assert transition.getRegionInfoCount() == 3 : transition;
+            final HRegionInfo parent = HRegionInfo.convert(transition.getRegionInfo(0));
+            final HRegionInfo splitA = HRegionInfo.convert(transition.getRegionInfo(1));
+            final HRegionInfo splitB = HRegionInfo.convert(transition.getRegionInfo(2));
+            updateRegionSplitTransition(serverName, transition.getTransitionCode(),
+              parent, splitA, splitB);
+            break;
+          case READY_TO_MERGE:
+          case MERGE_PONR:
+          case MERGED:
+          case MERGE_REVERTED:
+            assert transition.getRegionInfoCount() == 3 : transition;
+            final HRegionInfo merged = HRegionInfo.convert(transition.getRegionInfo(0));
+            final HRegionInfo mergeA = HRegionInfo.convert(transition.getRegionInfo(1));
+            final HRegionInfo mergeB = HRegionInfo.convert(transition.getRegionInfo(2));
+            updateRegionMergeTransition(serverName, transition.getTransitionCode(),
+              merged, mergeA, mergeB);
+            break;
+        }
+      }
+    } catch (PleaseHoldException e) {
+      LOG.debug("failed to transition: " + e.getMessage());
+      throw e;
+    } catch (UnsupportedOperationException|IOException e) {
+      // TODO: at the moment we have a single error message and the RS will abort
+      // if the master says that one of the region transition failed.
+      LOG.warn("failed to transition: " + e.getMessage());
+      builder.setErrorMessage("failed to transition: " + e.getMessage());
+    }
+    return builder.build();
+  }
+
+  private void updateRegionTransition(final ServerName serverName, final TransitionCode state,
+      final HRegionInfo regionInfo, final long seqId)
+      throws PleaseHoldException, UnexpectedStateException {
+    checkFailoverCleanupCompleted(regionInfo);
+
+    final RegionStateNode regionNode = regionStateMap.getRegionNode(regionInfo);
+    if (regionNode == null) {
+      // the table/region is gone. maybe a delete, split, merge
+      throw new UnexpectedStateException(String.format(
+        "Server %s was trying to transition region %s to %s. but the region was removed.",
+        serverName, regionInfo, state));
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("Update region transition serverName=%s region=%s state=%s",
+        serverName, regionNode, state));
+    }
+
+    final ServerStateNode serverNode = regionStateMap.getOrCreateServer(serverName);
+    if (!reportTransition(regionNode, serverNode, state, seqId)) {
+      LOG.warn(String.format(
+        "no procedure found for region=%s. server=%s was trying to transition to %s",
+        regionNode, serverName, state));
+    }
+  }
+
+  private boolean reportTransition(final RegionStateNode regionNode,
+      final ServerStateNode serverNode, final TransitionCode state, final long seqId)
+      throws UnexpectedStateException {
+    final ServerName serverName = serverNode.getServerName();
+    synchronized (regionNode) {
+      final RegionTransitionProcedure proc = regionNode.getProcedure();
+      if (proc == null) return false;
+
+      //serverNode.getReportEvent().removeProcedure(proc);
+      proc.reportTransition(master.getMasterProcedureExecutor().getEnvironment(),
+        serverName, state, seqId);
+      return true;
+    }
+  }
+
+  private void updateRegionSplitTransition(final ServerName serverName, final TransitionCode state,
+      final HRegionInfo parent, final HRegionInfo hriA, final HRegionInfo hriB)
+      throws IOException {
+    checkFailoverCleanupCompleted(parent);
+
+    if (state != TransitionCode.READY_TO_SPLIT) {
+      throw new UnexpectedStateException("unsupported split state=" + state +
+        " for parent region " + parent +
+        " maybe an old RS (< 2.0) had the operation in progress");
+    }
+
+    // sanity check on the request
+    if (!Bytes.equals(hriA.getEndKey(), hriB.getStartKey())) {
+      throw new UnsupportedOperationException(
+        "unsupported split request with bad keys: parent=" + parent +
+        " hriA=" + hriA + " hriB=" + hriB);
+    }
+
+    try {
+      regionStateListener.onRegionSplit(parent);
+    } catch (QuotaExceededException e) {
+      // TODO: does this really belong here?
+      master.getRegionNormalizer().planSkipped(parent, PlanType.SPLIT);
+      throw e;
+    }
+
+    // Submit the Split procedure
+    final byte[] splitKey = hriB.getStartKey();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("handling split request from RS=" + serverName +
+          ", parent=" + parent + " splitKey=" + Bytes.toStringBinary(splitKey));
+    }
+    master.getMasterProcedureExecutor().submitProcedure(createSplitProcedure(parent, splitKey));
+
+    // If the RS is < 2.0 throw an exception to abort the operation, we are handling the split
+    if (regionStateMap.getOrCreateServer(serverName).getVersionNumber() < 0x0200000) {
+      throw new UnsupportedOperationException(String.format(
+        "Split handled by the master: parent=%s hriA=%s hriB=%s", parent, hriA, hriB));
+    }
+  }
+
+  private void updateRegionMergeTransition(final ServerName serverName, final TransitionCode state,
+      final HRegionInfo merged, final HRegionInfo hriA, final HRegionInfo hriB)
+      throws PleaseHoldException, UnexpectedStateException, IOException {
+    checkFailoverCleanupCompleted(merged);
+
+    if (state != TransitionCode.READY_TO_MERGE) {
+      throw new UnexpectedStateException("Unsupported merge state=" + state +
+        " for regionA=" + hriA + " regionB=" + hriB + " merged=" + merged +
+        " maybe an old RS (< 2.0) had the operation in progress");
+    }
+
+    // Submit the Merge procedure
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Handling merge request from RS=" + merged + ", merged=" + merged);
+    }
+    master.getMasterProcedureExecutor().submitProcedure(createMergeProcedure(hriA, hriB));
+
+    // If the RS is < 2.0 throw an exception to abort the operation, we are handling the merge
+    if (regionStateMap.getOrCreateServer(serverName).getVersionNumber() < 0x0200000) {
+      throw new UnsupportedOperationException(String.format(
+        "Merge not handled yet: state=%s merged=%s hriA=%s hriB=%s", state, merged, hriA, hriB));
+    }
+  }
+
+  // ============================================================================================
+  //  RS Status update (report online regions) helpers
+  // ============================================================================================
+  /**
+   * the master will call this method when the RS send the regionServerReport().
+   * the report will contains the "hbase version" and the "online regions".
+   * this method will check the the online regions against the in-memory state of the AM,
+   * if there is a mismatch we will try to fence out the RS with the assumption
+   * that something went wrong on the RS side.
+   */
+  public void reportOnlineRegions(final ServerName serverName,
+      final int versionNumber, final Set<byte[]> regionNames) {
+    if (!isRunning()) return;
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("ReportOnlineRegions " + serverName + " regionCount=" + regionNames.size() +
+        ", metaLoaded=" + isMetaLoaded());
+    }
+
+    final ServerStateNode serverNode = regionStateMap.getOrCreateServer(serverName);
+
+    // update the server version number. This will be used for live upgrades.
+    synchronized (serverNode) {
+      serverNode.setVersionNumber(versionNumber);
+      if (serverNode.isInState(ServerState.SPLITTING, ServerState.OFFLINE)) {
+        LOG.warn("Got a report from a server result in state " + serverNode.getState());
+        return;
+      }
+    }
+
+    if (regionNames.isEmpty()) {
+      // nothing to do if we don't have regions
+      LOG.trace("no online region found on " + serverName);
+    } else if (!isMetaLoaded()) {
+      // if we are still on startup, discard the report unless is from someone holding meta
+      checkOnlineRegionsReportForMeta(serverNode, regionNames);
+    } else {
+      // The Heartbeat updates us of what regions are only. check and verify the state.
+      checkOnlineRegionsReport(serverNode, regionNames);
+    }
+
+    // wake report event
+    wakeServerReportEvent(serverNode);
+  }
+
+  public void checkOnlineRegionsReportForMeta(final ServerStateNode serverNode,
+      final Set<byte[]> regionNames) {
+    try {
+      for (byte[] regionName: regionNames) {
+        final HRegionInfo hri = getMetaRegionFromName(regionName);
+        if (hri == null) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Skip online report for region=" + Bytes.toStringBinary(regionName) +
+              " while meta is loading");
+          }
+          continue;
+        }
+
+        final RegionStateNode regionNode = regionStateMap.getOrCreateRegionNode(hri);
+        LOG.info("META REPORTED: " + regionNode);
+        if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
+          LOG.warn("META REPORTED but no procedure found");
+          regionNode.setRegionLocation(serverNode.getServerName());
+        } else if (LOG.isTraceEnabled()) {
+          LOG.trace("META REPORTED: " + regionNode);
+        }
+      }
+    } catch (UnexpectedStateException e) {
+      final ServerName serverName = serverNode.getServerName();
+      LOG.warn("Killing server=" + serverName + ": " + e.getMessage());
+      killRegionServer(serverNode);
+    }
+  }
+
+  public void checkOnlineRegionsReport(final ServerStateNode serverNode,
+      final Set<byte[]> regionNames) {
+    final ServerName serverName = serverNode.getServerName();
+    try {
+      for (byte[] regionName: regionNames) {
+        if (!isRunning()) return;
+
+        final RegionStateNode regionNode = regionStateMap.getRegionNodeFromName(regionName);
+        if (regionNode == null) {
+          throw new UnexpectedStateException(
+            "Reported online region " + Bytes.toStringBinary(regionName) + " not found");
+        }
+
+        synchronized (regionNode) {
+          if (regionNode.isInState(State.OPENING, State.OPEN)) {
+            if (!regionNode.getRegionLocation().equals(serverName)) {
+              throw new UnexpectedStateException(
+                "Reported OPEN region on server=" + serverName +
+                " but the state found says server=" + regionNode.getRegionLocation());
+            } else if (regionNode.isInState(State.OPENING)) {
+              try {
+                if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
+                  LOG.warn("Reported OPEN region on server=" + serverName +
+                    " but the state found says " + regionNode + " and no procedure is running");
+                }
+              } catch (UnexpectedStateException e) {
+                LOG.warn("unexpected exception while trying to report " + regionNode +
+                  " as open: " + e.getMessage(), e);
+              }
+            }
+          } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
+            // TODO: We end up killing the RS if we get a report while we already
+            // transitioned to close or split. we should have a timeout/timestamp to compare
+            throw new UnexpectedStateException(
+                "Reported OPEN region, but the state found says " + regionNode.getState());
+          }
+        }
+      }
+    } catch (UnexpectedStateException e) {
+      LOG.warn("Killing server=" + serverName + ": " + e.getMessage());
+      killRegionServer(serverNode);
+    }
+  }
+
+  protected boolean waitServerReportEvent(final ServerName serverName, final Procedure proc) {
+    final ServerStateNode serverNode = regionStateMap.getOrCreateServer(serverName);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Wait for " + serverName + " report on " + proc);
+    }
+    return getProcedureScheduler().waitEvent(serverNode.getReportEvent(), proc);
+  }
+
+  protected void wakeServerReportEvent(final ServerStateNode serverNode) {
+    getProcedureScheduler().wakeEvent(serverNode.getReportEvent());
+  }
+
+  // ============================================================================================
+  //  RIT chore
+  // ============================================================================================
+  private static class RegionInTransitionChore extends ProcedureInMemoryChore<MasterProcedureEnv> {
+    public RegionInTransitionChore(final int timeoutMsec) {
+      super(timeoutMsec);
+    }
+
+    @Override
+    protected void periodicExecute(final MasterProcedureEnv env) {
+      final AssignmentManager am = env.getAssignmentManager();
+
+      final RegionInTransitionStat ritStat = am.computeRegionInTransitionStat();
+      if (ritStat.hasRegionsOverThreshold()) {
+        for (RegionState hri: ritStat.getRegionOverThreshold()) {
+          am.handleRegionOverStuckWarningThreshold(hri.getRegion());
+        }
+      }
+
+      // update metrics
+      am.updateRegionsInTransitionMetrics(ritStat);
+    }
+  }
+
+  public RegionInTransitionStat computeRegionInTransitionStat() {
+    final RegionInTransitionStat rit = new RegionInTransitionStat(getConfiguration());
+    rit.update(this);
+    return rit;
+  }
+
+  public static class RegionInTransitionStat {
+    private final int ritThreshold;
+
+    private HashMap<String, RegionState> ritsOverThreshold = null;
+    private long statTimestamp;
+    private long oldestRITTime = 0;
+    private int totalRITsTwiceThreshold = 0;
+    private int totalRITs = 0;
+
+    protected RegionInTransitionStat(final Configuration conf) {
+      this.ritThreshold =
+        conf.getInt(METRICS_RIT_STUCK_WARNING_THRESHOLD, DEFAULT_RIT_STUCK_WARNING_THRESHOLD);
+    }
+
+    public int getRITThreshold() {
+      return ritThreshold;
+    }
+
+    public long getTimestamp() {
+      return statTimestamp;
+    }
+
+    public int getTotalRITs() {
+      return totalRITs;
+    }
+
+    public long getOldestRITTime() {
+      return oldestRITTime;
+    }
+
+    public int getTotalRITsOverThreshold() {
+      return ritsOverThreshold != null ? ritsOverThreshold.size() : 0;
+    }
+
+    public boolean hasRegionsTwiceOverThreshold() {
+      return totalRITsTwiceThreshold > 0;
+    }
+
+    public boolean hasRegionsOverThreshold() {
+      return ritsOverThreshold != null && !ritsOverThreshold.isEmpty();
+    }
+
+    public Collection<RegionState> getRegionOverThreshold() {
+      return ritsOverThreshold.values();
+    }
+
+    public boolean isRegionOverThreshold(final HRegionInfo regionInfo) {
+      return ritsOverThreshold.containsKey(regionInfo.getEncodedName());
+    }
+
+    public boolean isRegionTwiceOverThreshold(final HRegionInfo regionInfo) {
+      final RegionState state = ritsOverThreshold.get(regionInfo.getEncodedName());
+      if (state == null) return false;
+      return (statTimestamp - state.getStamp()) > (ritThreshold * 2);
+    }
+
+    protected void update(final AssignmentManager am) {
+      final RegionStates regionStates = am.getRegionStates();
+      this.statTimestamp = EnvironmentEdgeManager.currentTime();
+      update(regionStates.getRegionsStateInTransition(), statTimestamp);
+      update(regionStates.getRegionFailedOpen(), statTimestamp);
+    }
+
+    private void update(final Collection<RegionState> regions, final long currentTime) {
+      for (RegionState state: regions) {
+        totalRITs++;
+        final long ritTime = currentTime - state.getStamp();
+        if (ritTime > ritThreshold) {
+          if (ritsOverThreshold == null) {
+            ritsOverThreshold = new HashMap<String, RegionState>();
+          }
+          ritsOverThreshold.put(state.getRegion().getEncodedName(), state);
+          totalRITsTwiceThreshold += (ritTime > (ritThreshold * 2)) ? 1 : 0;
+        }
+        if (oldestRITTime < ritTime) {
+          oldestRITTime = ritTime;
+        }
+      }
+    }
+  }
+
+  private void updateRegionsInTransitionMetrics(final RegionInTransitionStat ritStat) {
+    metrics.updateRITOldestAge(ritStat.getOldestRITTime());
+    metrics.updateRITCount(ritStat.getTotalRITs());
+    metrics.updateRITCountOverThreshold(ritStat.getTotalRITsOverThreshold());
+  }
+
+  private void handleRegionOverStuckWarningThreshold(final HRegionInfo regionInfo) {
+    final RegionStateNode regionNode = regionStateMap.getRegionNode(regionInfo);
+    //if (regionNode.isStuck()) {
+    LOG.warn("TODO Handle region stuck in transition: " + regionNode);
+  }
+
+  // ============================================================================================
+  //  TODO: Master load/bootstrap
+  // ============================================================================================
+  public void joinCluster() throws IOException {
+    final long startTime = System.currentTimeMillis();
+
+    LOG.info("Joining the cluster...");
+
+    // Scan hbase:meta to build list of existing regions, servers, and assignment
+    loadMeta();
+
+    for (int i = 0; master.getServerManager().countOfRegionServers() < 1; ++i) {
+      LOG.info("waiting for RS to join");
+      Threads.sleep(250);
+    }
+    LOG.info("RS joined " + master.getServerManager().countOfRegionServers());
+
+    // This method will assign all user regions if a clean server startup or
+    // it will reconstruct master state and cleanup any leftovers from previous master process.
+    boolean failover = processofflineServersWithOnlineRegions();
+
+    // Start the RIT chore
+    master.getMasterProcedureExecutor().addChore(this.ritChore);
+
+    LOG.info(String.format("Joined the cluster in %s, failover=%s",
+      StringUtils.humanTimeDiff(System.currentTimeMillis() - startTime), failover));
+  }
+
+  private void loadMeta() throws IOException {
+    // TODO: use a thread pool
+    regionStateStore.visitMeta(new RegionStateStore.RegionStateVisitor() {
+      @Override
+      public void visitRegionState(final HRegionInfo regionInfo, final State state,
+          final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
+        final RegionStateNode regionNode = regionStateMap.getOrCreateRegionNode(regionInfo);
+        synchronized (regionNode) {
+          if (!regionNode.isInTransition()) {
+            regionNode.setState(state);
+            regionNode.setLastHost(lastHost);
+            regionNode.setRegionLocation(regionLocation);
+            regionNode.setOpenSeqNum(openSeqNum);
+
+            if (state == State.OPEN) {
+              assert regionLocation != null : "found null region location for " + regionNode;
+              regionStateMap.addRegionToServer(regionLocation, regionNode);
+            } else if (state == State.OFFLINE || regionInfo.isOffline()) {
+              regionStateMap.addToOfflineRegions(regionNode);
+            } else {
+              // These regions should have a procedure in replay
+              regionStateMap.addRegionInTransition(regionNode, null);
+            }
+          }
+        }
+      }
+    });
+
+    // every assignment is blocked until meta is loaded.
+    wakeMetaLoadedEvent();
+  }
+
+  // TODO: the assumption here is that if RSs are crashing while we are executing this
+  // they will be handled by the SSH that will be putted in the ServerManager "queue".
+  // we can integrate this a bit better.
+  private boolean processofflineServersWithOnlineRegions() {
+    boolean failover = !master.getServerManager().getDeadServers().isEmpty();
+
+    final Set<ServerName> offlineServersWithOnlineRegions = new HashSet<ServerName>();
+    final ArrayList<HRegionInfo> regionsToAssign = new ArrayList<HRegionInfo>();
+    long st, et;
+
+    st = System.currentTimeMillis();
+    for (RegionStateNode regionNode: regionStateMap.getRegionNodes()) {
+      if (regionNode.getState() == State.OPEN) {
+        final ServerName serverName = regionNode.getRegionLocation();
+        if (!master.getServerManager().isServerOnline(serverName)) {
+          offlineServersWithOnlineRegions.add(serverName);
+        }
+      } else if (regionNode.getState() == State.OFFLINE) {
+        if (isTableEnabled(regionNode.getTable())) {
+          regionsToAssign.add(regionNode.getRegionInfo());
+        }
+      }
+    }
+    et = System.currentTimeMillis();
+    LOG.info("[STEP-1] " + StringUtils.humanTimeDiff(et - st));
+
+    // kill servers with online regions
+    st = System.currentTimeMillis();
+    for (ServerName serverName: offlineServersWithOnlineRegions) {
+      if (!master.getServerManager().isServerOnline(serverName)) {
+        LOG.info("KILL RS hosting regions but not online " + serverName +
+          " (master=" + master.getServerName() + ")");
+        killRegionServer(serverName);
+      }
+    }
+    et = System.currentTimeMillis();
+    LOG.info("[STEP-2] " + StringUtils.humanTimeDiff(et - st));
+
+    setFailoverCleanupDone(true);
+
+    // assign offline regions
+    st = System.currentTimeMillis();
+    for (HRegionInfo regionInfo: regionsToAssign) {
+      master.getMasterProcedureExecutor().submitProcedure(
+        createAssignProcedure(regionInfo, false));
+    }
+    et = System.currentTimeMillis();
+    LOG.info("[STEP-3] " + StringUtils.humanTimeDiff(et - st));
+
+    return failover;
+  }
+
+  /**
+   * Used by ServerCrashProcedure to make sure AssignmentManager has completed
+   * the failover cleanup before re-assigning regions of dead servers. So that
+   * when re-assignment happens, AssignmentManager has proper region states.
+   */
+  public boolean isFailoverCleanupDone() {
+    return failoverCleanupDone.isReady();
+  }
+
+  /**
+   * Used by ServerCrashProcedure tests verify the ability to suspend the
+   * execution of the ServerCrashProcedure.
+   */
+  @VisibleForTesting
+  public void setFailoverCleanupDone(final boolean b) {
+    master.getMasterProcedureExecutor().getEnvironment()
+      .setEventReady(failoverCleanupDone, b);
+  }
+
+  public ProcedureEvent getFailoverCleanupEvent() {
+    return failoverCleanupDone;
+  }
+
+  /**
+   * Used to check if the failover cleanup is done.
+   * if not we throw PleaseHoldException since we are rebuilding the RegionStates
+   * @param hri region to check if it is already rebuild
+   * @throws PleaseHoldException if the failover cleanup is not completed
+   */
+  private void checkFailoverCleanupCompleted(final HRegionInfo hri) throws PleaseHoldException {
+    if (!isRunning()) {
+      throw new PleaseHoldException("AssignmentManager is not running");
+    }
+
+    // TODO: can we avoid throwing an exception if hri is already loaded?
+    //       at the moment we bypass only meta
+    if (!isMetaRegion(hri) && !isFailoverCleanupDone()) {
+      LOG.warn("Master is rebuilding user regions: " + hri);
+      throw new PleaseHoldException("Master is rebuilding user regions");
+    }
+  }
+
+  // ============================================================================================
+  //  TODO: Metrics
+  // ============================================================================================
+  public int getNumRegionsOpened() {
+    // TODO: Used by TestRegionPlacement.java and assume monotonically increasing value
+    return 0;
+  }
+
+  public void submitServerCrash(final ServerName serverName, final boolean shouldSplitWal) {
+    boolean carryingMeta = master.getAssignmentManager().isCarryingMeta(serverName);
+    ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
+    procExec.submitProcedure(new ServerCrashProcedure(procExec.getEnvironment(), serverName,
+      shouldSplitWal, carryingMeta));
+    LOG.debug("Added=" + serverName +
+      " to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
+  }
+
+  public void offlineRegion(final HRegionInfo regionInfo) throws IOException {
+    // TODO used by MasterRpcServices ServerCrashProcedure
+    LOG.info("OFFLINE REGION " + regionInfo);
+    final RegionStateNode node = regionStateMap.getRegionNode(regionInfo);
+    if (node != null) {
+      node.setState(State.OFFLINE);
+      node.setRegionLocation(null);
+    }
+  }
+
+  public void onlineRegion(final HRegionInfo regionInfo, final ServerName serverName) {
+    // TODO used by TestSplitTransactionOnCluster.java
+  }
+
+  public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
+      final Collection<HRegionInfo> regions) {
+    return regionStateMap.getSnapShotOfAssignment(regions);
+  }
+
+  // ============================================================================================
+  //  TODO: UTILS/HELPERS?
+  // ============================================================================================
+  /**
+   * Used by the client (via master) to identify if all regions have the schema updates
+   *
+   * @param tableName
+   * @return Pair indicating the status of the alter command (pending/total)
+   * @throws IOException
+   */
+  public Pair<Integer, Integer> getReopenStatus(TableName tableName)
+      throws IOException {
+    if (isTableDisabled(tableName)) return new Pair<Integer, Integer>(0, 0);
+
+    final List<RegionState> states = regionStateMap.getTableRegionStates(tableName);
+    int ritCount = 0;
+    for (RegionState regionState: states) {
+      if (!regionState.isOpened()) ritCount++;
+    }
+    return new Pair<Integer, Integer>(ritCount, states.size());
+  }
+
+  // ============================================================================================
+  //  TODO: Region State In Transition
+  // ============================================================================================
+  protected boolean addRegionInTransition(final RegionStateNode regionNode,
+      final RegionTransitionProcedure procedure) {
+    return regionStateMap.addRegionInTransition(regionNode, procedure);
+  }
+
+  protected void removeRegionInTransition(final RegionStateNode regionNode,
+      final RegionTransitionProcedure procedure) {
+    regionStateMap.removeRegionInTransition(regionNode, procedure);
+  }
+
+  public boolean hasRegionsInTransition() {
+    return regionStateMap.hasRegionsInTransition();
+  }
+
+  public List<RegionStateNode> getRegionsInTransition() {
+    return regionStateMap.getRegionsInTransition();
+  }
+
+  public List<HRegionInfo> getAssignedRegions() {
+    return regionStateMap.getAssignedRegions();
+  }
+
+  public HRegionInfo getRegionInfo(final byte[] regionName) {
+    final RegionStateNode regionState = regionStateMap.getRegionNodeFromName(regionName);
+    return regionState != null ? regionState.getRegionInfo() : null;
+  }
+
+  // ============================================================================================
+  //  TODO: Region Status update
+  // ============================================================================================
+  public void markRegionAsOpening(final RegionStateNode regionNode) throws IOException {
+    LOG.info("TODO: MARK REGION AS OPENING " + regionNode);
+    synchronized (regionNode) {
+      if (!regionNode.setState(State.OPENING, RegionStates.STATES_EXPECTEX_IN_OPEN)) {
+        throw new UnexpectedStateException(
+          "unexpected state " + regionNode.getState() + " for region " + regionNode);
+      }
+
+      // TODO: Do we need to update the state?
+      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), State.OPENING,
+        regionNode.getRegionLocation(), regionNode.getLastHost(), HConstants.NO_SEQNUM);
+    }
+
+    // update the operation count metrics
+    metrics.incrementOperationCounter();
+  }
+
+  public void markRegionAsOpened(final RegionStateNode regionNode) throws IOException {
+    final HRegionInfo hri = regionNode.getRegionInfo();
+    synchronized (regionNode) {
+      if (!regionNode.setState(State.OPEN, RegionStates.STATES_EXPECTEX_IN_OPEN)) {
+        throw new UnexpectedStateException(
+          "Unexpected state " + regionNode.getState() + " for region " + regionNode);
+      }
+
+      // TODO: Update Meta
+      if (isMetaRegion(hri)) {
+        setMetaInitialized(hri, true);
+      }
+
+      // TODO: This is done? St.Ack
+      LOG.info("TODO: MARK REGION AS OPEN " + regionNode);
+      regionStateMap.addRegionToServer(regionNode.getRegionLocation(), regionNode);
+      // This is an RPC to hbase:meta while lock on regionNode.
+      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), State.OPEN,
+        regionNode.getRegionLocation(), regionNode.getLastHost(), regionNode.getOpenSeqNum());
+
+      sendRegionOpenedNotification(hri, regionNode.getRegionLocation());
+
+      // update assignment metrics
+      if (regionNode.getProcedure() != null) {
+        metrics.updateAssignTime(regionNode.getProcedure().elapsedTime());
+      }
+    }
+  }
+
+  public void markRegionAsClosing(final RegionStateNode regionNode) throws IOException {
+    LOG.info("TODO: MARK REGION AS CLOSING " + regionNode);
+    final HRegionInfo hri = regionNode.getRegionInfo();
+    synchronized (regionNode) {
+      if (!regionNode.setState(State.CLOSING, RegionStates.STATES_EXPECTEX_IN_CLOSE)) {
+        throw new UnexpectedStateException(
+          "unexpected state " + regionNode.getState() + " for region " + regionNode);
+      }
+
+      // set meta has not initialized early. so people trying to create/edit tables will wait
+      if (isMetaRegion(hri)) {
+        setMetaInitialized(hri, false);
+      }
+
+      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), State.CLOSING,
+        regionNode.getRegionLocation(), regionNode.getLastHost(), HConstants.NO_SEQNUM);
+    }
+
+    // update the operation count metrics
+    metrics.incrementOperationCounter();
+  }
+
+  public void markRegionAsClosed(final RegionStateNode regionNode) throws IOException {
+    LOG.info("TODO: MARK REGION AS CLOSED " + regionNode);
+    final HRegionInfo hri = regionNode.getRegionInfo();
+    synchronized (regionNode) {
+      if (!regionNode.setState(State.CLOSED, RegionStates.STATES_EXPECTEX_IN_CLOSE)) {
+        throw new UnexpectedStateException(
+          "unexpected state " + regionNode.getState() + " for region " + regionNode);
+      }
+
+      regionStateMap.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
+      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), State.CLOSED,
+        regionNode.getRegionLocation(), regionNode.getLastHost(), HConstants.NO_SEQNUM);
+
+      sendRegionClosedNotification(hri);
+
+      // update assignment metrics
+      if (regionNode.getProcedure() != null) {
+        metrics.updateUnassignTime(regionNode.getProcedure().elapsedTime());
+      }
+    }
+  }
+
+  private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
+      final ServerName serverName) {
+    getBalancer().regionOnline(regionInfo, serverName);
+    if (!this.listeners.isEmpty()) {
+      for (AssignmentListener listener : this.listeners) {
+        listener.regionOpened(regionInfo, serverName);
+      }
+    }
+  }
+
+  private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
+    getBalancer().regionOffline(regionInfo);
+    if (!this.listeners.isEmpty()) {
+      for (AssignmentListener listener : this.listeners) {
+        listener.regionClosed(regionInfo);
+      }
+    }
+  }
+
+  public void markRegionAsSplitted(final HRegionInfo parent, final ServerName serverName,
+      final HRegionInfo hriA, final HRegionInfo hriB) throws IOException {
+    parent.setSplit(true);
+    final RegionStateNode node = regionStateMap.getOrCreateRegionNode(parent);
+    node.getRegionInfo().setSplit(true);
+    node.getRegionInfo().setOffline(true);
+    node.setState(State.SPLIT);
+
+    LOG.info("TODO: MARK REGION AS SPLIT parent=" + parent);
+    regionStateStore.splitRegion(parent, hriA, hriB, serverName);
+  }
+
+  public void markRegionAsMerged(final HRegionInfo merged, final ServerName serverName,
+      final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB) throws IOException {
+    final RegionStateNode node = regionStateMap.getOrCreateRegionNode(merged);
+    node.getRegionInfo().setOffline(true);
+    node.setState(State.MERGED);
+
+    LOG.info("TODO: MARK REGION AS MERGED parent=" + merged);
+    regionStateStore.mergeRegions(merged, regionToMergeA, regionToMergeB, serverName);
+  }
+
+  // ============================================================================================
+  //  Assign Queue (Assign/Balance)
+  // ============================================================================================
+  private final ArrayList<RegionStateNode> pendingAssignQueue = new ArrayList<RegionStateNode>();
+  private final ReentrantLock assignQueueLock = new ReentrantLock();
+  private final Condition assignQueueFullCond = assignQueueLock.newCondition();
+
+  /**
+   * Add the assign operation to the assignment queue.
+   * The pending assignment operation will be processed,
+   * and each region will be assigned by a server using the balancer.
+   */
+  protected void queueAssign(final RegionStateNode regionNode) {
+    getProcedureScheduler().suspendEvent(regionNode.getProcedureEvent());
+
+    // TODO: quick-start for meta and the other sys-tables?
+    assignQueueLock.lock();
+    try {
+      pendingAssignQueue.add(regionNode);
+      if (regionNode.isSystemTable() ||
+          pendingAssignQueue.size() == 1 ||
+          pendingAssignQueue.size() >= assignDispatchWaitQueueMaxSize) {
+        assignQueueFullCond.signal();
+      }
+    } finally {
+      assignQueueLock.unlock();
+    }
+  }
+
+  private void startAssignmentThread() {
+    assignThread = new Thread("AssignmentThread") {
+      @Override
+      public void run() {
+        while (isRunning()) {
+          processAssignQueue();
+        }
+        pendingAssignQueue.clear();
+      }
+    };
+    assignThread.start();
+  }
+
+  private void stopAssignmentThread() {
+    assignQueueSignal();
+    try {
+      while (assignThread.isAlive()) {
+        assignQueueSignal();
+        assignThread.join(250);
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("join interrupted", e);
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  private void assignQueueSignal() {
+    assignQueueLock.lock();
+    try {
+      assignQueueFullCond.signal();
+    } finally {
+      assignQueueLock.unlock();
+    }
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
+  private HashMap<HRegionInfo, RegionStateNode> waitOnAssignQueue() {
+    HashMap<HRegionInfo, RegionStateNode> regions = null;
+
+    assignQueueLock.lock();
+    try {
+      if (pendingAssignQueue.isEmpty() && isRunning()) {
+        assignQueueFullCond.await();
+      }
+
+      if (!isRunning()) return null;
+      assignQueueFullCond.await(assignDispatchWaitMillis, TimeUnit.MILLISECONDS);
+      regions = new HashMap<HRegionInfo, RegionStateNode>(pendingAssignQueue.size());
+      for (RegionStateNode regionNode: pendingAssignQueue) {
+        regions.put(regionNode.getRegionInfo(), regionNode);
+      }
+      pendingAssignQueue.clear();
+    } catch (InterruptedException e) {
+      LOG.warn("got interrupted ", e);
+      Thread.currentThread().interrupt();
+    } finally {
+      assignQueueLock.unlock();
+    }
+    return regions;
+  }
+
+  private void processAssignQueue() {
+    final HashMap<HRegionInfo, RegionStateNode> regions = waitOnAssignQueue();
+    if (regions == null || regions.size() == 0 || !isRunning()) {
+      return;
+    }
+
+    LOG.info("PROCESS ASSIGN QUEUE regionCount=" + regions.size());
+
+    // TODO: Optimize balancer. pass a RegionPlan?
+    final HashMap<HRegionInfo, ServerName> retainMap = new HashMap<HRegionInfo, ServerName>();
+    final List<HRegionInfo> rrList = new ArrayList<HRegionInfo>();
+    for (RegionStateNode regionNode: regions.values()) {
+      if (regionNode.getRegionLocation() != null) {
+        retainMap.put(regionNode.getRegionInfo(), regionNode.getRegionLocation());
+      } else {
+        rrList.add(regionNode.getRegionInfo());
+      }
+    }
+
+    // TODO: connect with the listener to invalidate the cache
+    final LoadBalancer balancer = getBalancer();
+
+    // TODO use events
+    List<ServerName> servers = master.getServerManager().createDestinationServersList();
+    for (int i = 0; servers.size() < 1; ++i) {
+      if (i % 4 == 0) {
+        LOG.warn("no server available, unable to find a location for " + regions.size() +
+            " unassigned regions. waiting");
+      }
+
+      // the was AM killed
+      if (!isRunning()) {
+        LOG.debug("aborting assignment-queue with " + regions.size() + " not assigned");
+        return;
+      }
+
+      Threads.sleep(250);
+      servers = master.getServerManager().createDestinationServersList();
+    }
+
+    final boolean isTraceEnabled = LOG.isTraceEnabled();
+    if (isTraceEnabled) {
+      LOG.trace("available servers count=" + servers.size() + ": " + servers);
+    }
+
+    // ask the balancer where to place regions
+    if (!retainMap.isEmpty()) {
+      if (isTraceEnabled) {
+        LOG.trace("retain assign regions=" + retainMap);
+      }
+      try {
+        acceptPlan(regions, balancer.retainAssignment(retainMap, servers));
+      } catch (HBaseIOException e) {
+        LOG.warn("unable to retain assignment", e);
+        addToPendingAssignment(regions, retainMap.keySet());
+      }
+    }
+
+    // TODO: Do we need to split retain and round-robin?
+    // the retain seems to fallback to round-robin/random if the region is not in the map.
+    if (!rrList.isEmpty()) {
+      Collections.sort(rrList);
+      if (isTraceEnabled) {
+        LOG.trace("round robin regions=" + rrList);
+      }
+      try {
+        acceptPlan(regions, balancer.roundRobinAssignment(rrList, servers));
+      } catch (HBaseIOException e) {
+        LOG.warn("unable to round-robin assignment", e);
+        addToPendingAssignment(regions, rrList);
+      }
+    }
+  }
+
+  private void acceptPlan(final HashMap<HRegionInfo, RegionStateNode> regions,
+      final Map<ServerName, List<HRegionInfo>> plan) throws HBaseIOException {
+    final ProcedureEvent[] events = new ProcedureEvent[regions.size()];
+    final long st = System.currentTimeMillis();
+
+    if (plan == null) {
+      throw new HBaseIOException("unable to compute plans for regions=" + regions.size());
+    }
+
+    if (plan.isEmpty()) return;
+
+    int evcount = 0;
+    for (Map.Entry<ServerName, List<HRegionInfo>> entry: plan.entrySet()) {
+      final ServerName server = entry.getKey();
+      for (HRegionInfo hri: entry.getValue()) {
+        final RegionStateNode regionNode = regions.get(hri);
+        regionNode.setRegionLocation(server);
+        events[evcount++] = regionNode.getProcedureEvent();
+      }
+    }
+    getProcedureScheduler().wakeEvents(evcount, events);
+
+    final long et = System.currentTimeMillis();
+    LOG.info("ASSIGN ACCEPT " + events.length + " -> " + StringUtils.humanTimeDiff(et - st));
+  }
+
+  private void addToPendingAssignment(final HashMap<HRegionInfo, RegionStateNode> regions,
+      final Collection<HRegionInfo> pendingRegions) {
+    assignQueueLock.lock();
+    try {
+      for (HRegionInfo hri: pendingRegions) {
+        pendingAssignQueue.add(regions.get(hri));
+      }
+    } finally {
+      assignQueueLock.unlock();
+    }
+  }
+
+  // ============================================================================================
+  //  Server Helpers
+  // ============================================================================================
+  @Override
+  public void serverAdded(final ServerName serverName) {
+  }
+
+  @Override
+  public void serverRemoved(final ServerName serverName) {
+    final ServerStateNode serverNode = regionStateMap.getServerNode(serverName);
+    if (serverNode == null) return;
+
+    // just in case, wake procedures waiting for this server report
+    wakeServerReportEvent(serverNode);
+  }
+
+  public int getServerVersion(final ServerName serverName) {
+    final ServerStateNode node = regionStateMap.getServerNode(serverName);
+    return node != null ? node.getVersionNumber() : 0;
+  }
+
+  public void killRegionServer(final ServerName serverName) {
+    final ServerStateNode serverNode = regionStateMap.getServerNode(serverName);
+    killRegionServer(serverNode);
+  }
+
+  public void killRegionServer(final ServerStateNode serverNode) {
+    for (RegionStateNode regionNode: serverNode.getRegions()) {
+      regionNode.setState(State.OFFLINE);
+      regionNode.setRegionLocation(null);
+    }
+    master.getServerManager().expireServer(serverNode.getServerName());
+  }
+}
\ No newline at end of file


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index d87c71b..77c2d1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -695,7 +695,8 @@ public class WALSplitter {
    */
   public static long writeRegionSequenceIdFile(final FileSystem fs, final Path regiondir,
       long newSeqId, long saftyBumper) throws IOException {
-
+    // TODO: Why are we using a method in here as part of our normal region open where
+    // there is no splitting involved? Fix. St.Ack 01/20/2017.
     Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
     long maxSeqId = 0;
     FileStatus[] files = null;
@@ -732,7 +733,7 @@ public class WALSplitter {
           throw new IOException("Failed to create SeqId file:" + newSeqIdFile);
         }
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Wrote region seqId=" + newSeqIdFile + " to file, newSeqId=" + newSeqId
+          LOG.debug("Wrote file=" + newSeqIdFile + ", newSeqId=" + newSeqId
               + ", maxSeqId=" + maxSeqId);
         }
       } catch (FileAlreadyExistsException ignored) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 696ea18..20a72e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -88,9 +88,9 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.mapreduce.MapreduceTestingShim;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -3993,8 +3993,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
         if (master == null) return false;
         AssignmentManager am = master.getAssignmentManager();
         if (am == null) return false;
-        final RegionStates regionStates = am.getRegionStates();
-        return !regionStates.isRegionsInTransition();
+        return !am.hasRegionsInTransition();
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index 81b3489..0f23fea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -301,16 +301,6 @@ public class MockRegionServerServices implements RegionServerServices {
   }
 
   @Override
-  public long requestRegionSplit(final HRegionInfo regionInfo, final byte[] splitRow) {
-    return -1;
-  }
-
-  @Override
-  public boolean isProcedureFinished(final long procId) {
-    return false;
-  }
-
-  @Override
   public boolean registerService(Service service) {
     // TODO Auto-generated method stub
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
index 283d79d..93feb5a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -21,13 +21,18 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
@@ -43,12 +48,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
 /**
  * Test whether region re-balancing works. (HBASE-71)
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
index c4abd89..7ad99c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestStochasticBalancerJmxMetrics.java
@@ -127,7 +127,7 @@ public class TestStochasticBalancerJmxMetrics extends BalancerTestBase {
     conf.setBoolean(HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, false);
     loadBalancer.setConf(conf);
 
-    TableName tableName = TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME);
+    TableName tableName = HConstants.ENSEMBLE_TABLE_NAME;
     Map<ServerName, List<HRegionInfo>> clusterState = mockClusterServers(mockCluster_ensemble);
     loadBalancer.balanceCluster(tableName, clusterState);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 7b69db4..f84d9c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -42,23 +42,18 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
-import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -103,7 +98,7 @@ public class TestAdmin1 {
 
   @Before
   public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getHBaseAdmin();
+    this.admin = TEST_UTIL.getAdmin();
   }
 
   @After
@@ -751,7 +746,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getHBaseAdmin();
+    admin = TEST_UTIL.getAdmin();
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
@@ -806,7 +801,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getHBaseAdmin();
+    admin = TEST_UTIL.getAdmin();
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
 
@@ -992,7 +987,7 @@ public class TestAdmin1 {
       sb.append("_").append(Integer.toString(rowCounts[i]));
     }
     assertFalse(admin.tableExists(tableName));
-    try(final Table table = TEST_UTIL.createTable(tableName, familyNames,
+    try (final Table table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
       final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
 
@@ -1057,8 +1052,7 @@ public class TestAdmin1 {
             } catch (IOException e) {
               e.printStackTrace();
             }
-            if (regions == null)
-              continue;
+            if (regions == null) continue;
             count.set(regions.size());
             if (count.get() >= 2) {
               LOG.info("Found: " + regions);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 0014401..607fc61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -528,8 +528,6 @@ public class TestAdmin2 {
     List<HRegionInfo> tableRegions = localAdmin.getTableRegions(tableName);
     HRegionInfo hri = tableRegions.get(0);
     AssignmentManager am = master.getAssignmentManager();
-    assertTrue("Region " + hri.getRegionNameAsString()
-      + " should be assigned properly", am.waitForAssignment(hri));
     ServerName server = am.getRegionStates().getRegionServerOfRegion(hri);
     localAdmin.move(hri.getEncodedNameAsBytes(), Bytes.toBytes(server.getServerName()));
     assertEquals("Current region server and region server before move should be same.", server,

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
index 80b7208..e1277fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
@@ -18,14 +18,15 @@
  */
 package org.apache.hadoop.hbase.client;
 
+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.ArrayList;
 import java.util.List;
-import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -36,16 +37,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -57,9 +53,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 @Category({ MasterTests.class, MediumTests.class })
 public class TestEnableTable {
@@ -105,15 +101,17 @@ public class TestEnableTable {
     rs.getRegionServer().stop("stop");
     cluster.waitForRegionServerToStop(rs.getRegionServer().getServerName(), 10000);
 
-    LOG.debug("Now enabling table " + tableName);
-
-    admin.enableTable(tableName);
-    assertTrue(admin.isTableEnabled(tableName));
+    // We used to enable the table here but AMv2 would hang waiting on a RS to check-in.
+    // Revisit.
 
     JVMClusterUtil.RegionServerThread rs2 = cluster.startRegionServer();
     cluster.waitForRegionServerToStart(rs2.getRegionServer().getServerName().getHostname(),
         rs2.getRegionServer().getServerName().getPort(), 60000);
 
+    LOG.debug("Now enabling table " + tableName);
+    admin.enableTable(tableName);
+    assertTrue(admin.isTableEnabled(tableName));
+
     List<HRegionInfo> regions = TEST_UTIL.getAdmin().getTableRegions(tableName);
     assertEquals(1, regions.size());
     for (HRegionInfo region : regions) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index bfe10b5..3e394a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -60,8 +60,8 @@ import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.ipc.ServerTooBusyException;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
@@ -231,8 +231,6 @@ public class TestHCM {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY);
     // simulate queue blocking in testDropTimeoutRequest
     TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 1);
-    // Used in testServerBusyException
-    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD, 3);
     TEST_UTIL.startMiniCluster(2);
   }
 
@@ -1036,7 +1034,7 @@ public class TestHCM {
     Assert.assertNotNull(curServer.getOnlineRegion(regionName));
     Assert.assertNull(destServer.getOnlineRegion(regionName));
     Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
-        getAssignmentManager().getRegionStates().isRegionsInTransition());
+        getAssignmentManager().hasRegionsInTransition());
 
     // Moving. It's possible that we don't have all the regions online at this point, so
     //  the test must depends only on the region we're looking at.
@@ -1049,7 +1047,7 @@ public class TestHCM {
     while (destServer.getOnlineRegion(regionName) == null ||
         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+        master.getAssignmentManager().hasRegionsInTransition()) {
       // wait for the move to be finished
       Thread.sleep(1);
     }
@@ -1108,7 +1106,7 @@ public class TestHCM {
     while (curServer.getOnlineRegion(regionName) == null ||
         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+        master.getAssignmentManager().hasRegionsInTransition()) {
       // wait for the move to be finished
       Thread.sleep(1);
     }
@@ -1353,7 +1351,7 @@ public class TestHCM {
       Assert.assertNotNull(curServer.getOnlineRegion(regionName));
       Assert.assertNull(destServer.getOnlineRegion(regionName));
       Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
-          getAssignmentManager().getRegionStates().isRegionsInTransition());
+          getAssignmentManager().hasRegionsInTransition());
 
        // Moving. It's possible that we don't have all the regions online at this point, so
       //  the test must depends only on the region we're looking at.
@@ -1366,7 +1364,7 @@ public class TestHCM {
       while (destServer.getOnlineRegion(regionName) == null ||
           destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
           curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-          master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+          master.getAssignmentManager().hasRegionsInTransition()) {
         // wait for the move to be finished
         Thread.sleep(1);
       }
@@ -1478,107 +1476,4 @@ public class TestHCM {
     table.close();
     connection.close();
   }
-
-  private class TestPutThread extends Thread {
-    Table table;
-    int getServerBusyException = 0;
-
-    TestPutThread(Table table){
-      this.table = table;
-    }
-
-    @Override
-    public void run() {
-      try {
-        Put p = new Put(ROW);
-        p.addColumn(FAM_NAM, new byte[]{0}, new byte[]{0});
-        table.put(p);
-      } catch (RetriesExhaustedWithDetailsException e) {
-        if (e.exceptions.get(0) instanceof ServerTooBusyException) {
-          getServerBusyException = 1;
-        }
-      } catch (IOException ignore) {
-      }
-    }
-  }
-
-  private class TestGetThread extends Thread {
-    Table table;
-    int getServerBusyException = 0;
-
-    TestGetThread(Table table){
-      this.table = table;
-    }
-
-    @Override
-    public void run() {
-      try {
-        Get g = new Get(ROW);
-        g.addColumn(FAM_NAM, new byte[] { 0 });
-        table.get(g);
-      } catch (ServerTooBusyException e) {
-        getServerBusyException = 1;
-      } catch (IOException ignore) {
-      }
-    }
-  }
-
-  @Test()
-  public void testServerBusyException() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()));
-    hdt.addCoprocessor(SleepCoprocessor.class.getName());
-    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
-    TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c);
-
-    TestGetThread tg1 =
-        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestGetThread tg2 =
-        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestGetThread tg3 =
-        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestGetThread tg4 =
-        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestGetThread tg5 =
-        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    tg1.start();
-    tg2.start();
-    tg3.start();
-    tg4.start();
-    tg5.start();
-    tg1.join();
-    tg2.join();
-    tg3.join();
-    tg4.join();
-    tg5.join();
-    assertEquals(2,
-        tg1.getServerBusyException + tg2.getServerBusyException + tg3.getServerBusyException
-            + tg4.getServerBusyException + tg5.getServerBusyException);
-
-    // Put has its own logic in HTable, test Put alone. We use AsyncProcess for Put (use multi at
-    // RPC level) and it wrap exceptions to RetriesExhaustedWithDetailsException.
-
-    TestPutThread tp1 =
-        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestPutThread tp2 =
-        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestPutThread tp3 =
-        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestPutThread tp4 =
-        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    TestPutThread tp5 =
-        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
-    tp1.start();
-    tp2.start();
-    tp3.start();
-    tp4.start();
-    tp5.start();
-    tp1.join();
-    tp2.join();
-    tp3.join();
-    tp4.join();
-    tp5.join();
-    assertEquals(2,
-        tp1.getServerBusyException + tp2.getServerBusyException + tp3.getServerBusyException
-            + tp4.getServerBusyException + tp5.getServerBusyException);
-  }
-}
+}
\ 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/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 527c910..b5916b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsckRepair;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -449,9 +450,6 @@ public class TestMetaWithReplicas {
     // Create a meta replica (this will be the 4th one) and assign it
     HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
         HRegionInfo.FIRST_META_REGIONINFO, 3);
-    // create in-memory state otherwise master won't assign
-    TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
-             .getRegionStates().createRegionState(h);
     TEST_UTIL.assignRegion(h);
     HBaseFsckRepair.waitUntilAssigned(TEST_UTIL.getAdmin(), h);
     // check that problem exists

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index e5c19ac..c42ca6a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -39,9 +39,6 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -629,11 +626,8 @@ public class TestScannersFromClientSide {
     }
 
     // Now open the region again.
-    HMaster master = cluster.getMaster();
-    RegionStates states = master.getAssignmentManager().getRegionStates();
-    states.regionOffline(hri);
-    states.updateRegionState(hri, State.OPENING);
-    ProtobufUtil.openRegion(null, rs.getRSRpcServices(), rs.getServerName(), hri);
+    cluster.getMaster().getAssignmentManager().reopen(hri);
+
     startTime = EnvironmentEdgeManager.currentTime();
     while (true) {
       if (rs.getOnlineRegion(regionName) != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
new file mode 100644
index 0000000..c318ffc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
@@ -0,0 +1,234 @@
+/*
+ *
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.ipc.ServerTooBusyException;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Threads;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+
+/**
+ * This class is for testing HBaseConnectionManager ServerBusyException.
+ * Be careful adding to this class. It sets a low
+ * HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD
+ */
+@Category({LargeTests.class})
+public class TestServerBusyException {
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder()
+      .withTimeout(this.getClass())
+      .withLookingForStuckThread(true)
+      .build();
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final byte[] FAM_NAM = Bytes.toBytes("f");
+  private static final byte[] ROW = Bytes.toBytes("bbb");
+  private static final int RPC_RETRY = 5;
+
+  @Rule
+  public TestName name = new TestName();
+
+  public static class SleepCoprocessor implements RegionObserver {
+    public static final int SLEEP_TIME = 5000;
+    @Override
+    public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Get get, final List<Cell> results) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+    }
+
+    @Override
+    public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Put put, final WALEdit edit, final Durability durability) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+    }
+
+    @Override
+    public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> e,
+                               final Increment increment) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+      return null;
+    }
+
+    @Override
+    public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> e, final Delete delete,
+        final WALEdit edit, final Durability durability) throws IOException {
+      Threads.sleep(SLEEP_TIME);
+    }
+
+  }
+
+  public static class SleepLongerAtFirstCoprocessor implements RegionObserver {
+    public static final int SLEEP_TIME = 2000;
+    static final AtomicLong ct = new AtomicLong(0);
+    @Override
+    public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+        final Get get, final List<Cell> results) throws IOException {
+      // After first sleep, all requests are timeout except the last retry. If we handle
+      // all the following requests, finally the last request is also timeout. If we drop all
+      // timeout requests, we can handle the last request immediately and it will not timeout.
+      if (ct.incrementAndGet() <= 1) {
+        Threads.sleep(SLEEP_TIME * RPC_RETRY * 2);
+      } else {
+        Threads.sleep(SLEEP_TIME);
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
+    // Up the handlers; this test needs more than usual.
+    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, RPC_RETRY);
+    // simulate queue blocking in testDropTimeoutRequest
+    TEST_UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 1);
+    // Needed by the server busy test.
+    TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_PERSERVER_REQUESTS_THRESHOLD, 3);
+    TEST_UTIL.startMiniCluster(2);
+  }
+
+  @AfterClass public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private class TestPutThread extends Thread {
+    Table table;
+    int getServerBusyException = 0;
+
+    TestPutThread(Table table){
+      this.table = table;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Put p = new Put(ROW);
+        p.addColumn(FAM_NAM, new byte[]{0}, new byte[]{0});
+        table.put(p);
+      } catch (RetriesExhaustedWithDetailsException e) {
+        if (e.exceptions.get(0) instanceof ServerTooBusyException) {
+          getServerBusyException = 1;
+        }
+      } catch (IOException ignore) {
+      }
+    }
+  }
+
+  private class TestGetThread extends Thread {
+    Table table;
+    int getServerBusyException = 0;
+
+    TestGetThread(Table table){
+      this.table = table;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Get g = new Get(ROW);
+        g.addColumn(FAM_NAM, new byte[] { 0 });
+        table.get(g);
+      } catch (ServerTooBusyException e) {
+        getServerBusyException = 1;
+      } catch (IOException ignore) {
+      }
+    }
+  }
+
+  @Test()
+  public void testServerBusyException() throws Exception {
+    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()));
+    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    Configuration c = new Configuration(TEST_UTIL.getConfiguration());
+    TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c);
+
+    TestGetThread tg1 =
+        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestGetThread tg2 =
+        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestGetThread tg3 =
+        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestGetThread tg4 =
+        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestGetThread tg5 =
+        new TestGetThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    tg1.start();
+    tg2.start();
+    tg3.start();
+    tg4.start();
+    tg5.start();
+    tg1.join();
+    tg2.join();
+    tg3.join();
+    tg4.join();
+    tg5.join();
+    assertEquals(2,
+        tg1.getServerBusyException + tg2.getServerBusyException + tg3.getServerBusyException
+            + tg4.getServerBusyException + tg5.getServerBusyException);
+
+    // Put has its own logic in HTable, test Put alone. We use AsyncProcess for Put (use multi at
+    // RPC level) and it wrap exceptions to RetriesExhaustedWithDetailsException.
+
+    TestPutThread tp1 =
+        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestPutThread tp2 =
+        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestPutThread tp3 =
+        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestPutThread tp4 =
+        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    TestPutThread tp5 =
+        new TestPutThread(TEST_UTIL.getConnection().getTable(hdt.getTableName()));
+    tp1.start();
+    tp2.start();
+    tp3.start();
+    tp4.start();
+    tp5.start();
+    tp1.join();
+    tp2.join();
+    tp3.join();
+    tp4.join();
+    tp5.join();
+    assertEquals(2,
+        tp1.getServerBusyException + tp2.getServerBusyException + tp3.getServerBusyException
+            + tp4.getServerBusyException + tp5.getServerBusyException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
index f587d20..fe6e0c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java
@@ -43,11 +43,12 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -235,11 +236,13 @@ public class TestTableFavoredNodes {
     LOG.info("regionA: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionA));
     LOG.info("regionB: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionB));
 
+    int countOfRegions = MetaTableAccessor.getRegionCount(TEST_UTIL.getConfiguration(), tableName);
     admin.mergeRegionsAsync(regionA.getEncodedNameAsBytes(),
         regionB.getEncodedNameAsBytes(), false).get(60, TimeUnit.SECONDS);
 
     TEST_UTIL.waitUntilNoRegionsInTransition(WAIT_TIMEOUT);
-    waitUntilTableRegionCountReached(tableName, NUM_REGIONS - 1);
+    int howManyRegions = MetaTableAccessor.getRegionCount(TEST_UTIL.getConfiguration(), tableName);
+    waitUntilTableRegionCountReached(tableName, countOfRegions - 1);
 
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
index 35ed531..8805337 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestIncrementTimeRange.java
@@ -32,7 +32,6 @@ import java.util.NavigableMap;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -84,6 +84,9 @@ public class TestIncrementTimeRange {
   public static void setupBeforeClass() throws Exception {
     util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
         MyObserver.class.getName());
+    // Make general delay zero rather than default. Timing is off in this
+    // test that depends on an evironment edge that is manually moved forward.
+    util.getConfiguration().setInt(RemoteProcedureDispatcher.DISPATCH_DELAY_CONF_KEY, 0);
     util.startMiniCluster();
     EnvironmentEdgeManager.injectEdge(mee);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
index 97f74af..f49fd75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
@@ -150,6 +150,9 @@ public class TestChangingEncoding {
       Result result = table.get(get);
       for (int j = 0; j < NUM_COLS_PER_ROW; ++j) {
         Cell kv = result.getColumnLatestCell(CF_BYTES, getQualifier(j));
+        if (kv == null) {
+          continue;
+        }
         assertTrue(CellUtil.matchingValue(kv, getValue(batchId, i, j)));
       }
     }
@@ -238,7 +241,7 @@ public class TestChangingEncoding {
   public void testCrazyRandomChanges() throws Exception {
     prepareTest("RandomChanges");
     Random rand = new Random(2934298742974297L);
-    for (int i = 0; i < 20; ++i) {
+    for (int i = 0; i < 10; ++i) {
       int encodingOrdinal = rand.nextInt(DataBlockEncoding.values().length);
       DataBlockEncoding encoding = DataBlockEncoding.values()[encodingOrdinal];
       setEncodingConf(encoding, rand.nextBoolean());
@@ -246,5 +249,4 @@ public class TestChangingEncoding {
       verifyAllData();
     }
   }
-
-}
+}
\ 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/ipc/TestSimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
index 5e4520d..934125f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java
@@ -50,12 +50,12 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandlerImpl;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
 import org.apache.hadoop.hbase.testclassification.RPCTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -76,11 +76,11 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 
 @Category({RPCTests.class, SmallTests.class})
-public class TestSimpleRpcScheduler {/*
+public class TestSimpleRpcScheduler {
   @Rule
   public final TestRule timeout =
       CategoryBasedTimeout.builder().withTimeout(this.getClass()).
-          withLookingForStuckThread(true).build();*/
+          withLookingForStuckThread(true).build();
 
   private static final Log LOG = LogFactory.getLog(TestSimpleRpcScheduler.class);
 
@@ -434,7 +434,7 @@ public class TestSimpleRpcScheduler {/*
   @Test
   public void testCoDelScheduling() throws Exception {
     CoDelEnvironmentEdge envEdge = new CoDelEnvironmentEdge();
-    envEdge.threadNamePrefixs.add("RpcServer.deafult.FPBQ.Codel.handler");
+    envEdge.threadNamePrefixs.add("RpcServer.default.FPBQ.Codel.handler");
     Configuration schedConf = HBaseConfiguration.create();
     schedConf.setInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 250);
     schedConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
@@ -456,7 +456,6 @@ public class TestSimpleRpcScheduler {/*
       for (int i = 0; i < 100; i++) {
         long time = System.currentTimeMillis();
         envEdge.timeQ.put(time);
-        long now = System.currentTimeMillis();
         CallRunner cr = getMockedCallRunner(time, 2);
         // LOG.info("" + i + " " + (System.currentTimeMillis() - now) + " cr=" + cr);
         scheduler.dispatch(cr);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 683e9b3..1761df6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
@@ -44,7 +46,6 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.mockito.Mockito;
@@ -371,6 +372,11 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
+  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
+    return false;
+  }
+
+  @Override
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException {
   }
@@ -423,4 +429,10 @@ public class MockNoopMasterServices implements MasterServices, Server {
   public LockManager getLockManager() {
     return null;
   }
+
+  @Override
+  public long dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible, long nonceGroup,
+      long nonce) throws IOException {
+    return 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 78c8214..878dc5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
@@ -52,10 +51,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
@@ -68,6 +67,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerIn
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetStoreFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.MergeRegionsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
@@ -491,13 +492,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public CloseRegionForSplitOrMergeResponse closeRegionForSplitOrMerge(
-      RpcController controller,
-      CloseRegionForSplitOrMergeRequest request) throws ServiceException {
-    return null;
-  }
-
-  @Override
   public FlushRegionResponse flushRegion(RpcController controller,
       FlushRegionRequest request) throws ServiceException {
     // TODO Auto-generated method stub
@@ -505,16 +499,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public long requestRegionSplit(HRegionInfo regionInfo, byte[] splitRow) {
-    return -1;
-  }
-
-  @Override
-  public boolean isProcedureFinished(final long procId) {
-    return false;
-  }
-
-  @Override
   public SplitRegionResponse splitRegion(RpcController controller,
       SplitRegionRequest request) throws ServiceException {
     return null;
@@ -720,4 +704,17 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   @Override
   public void unassign(byte[] regionName) throws IOException {
   }
+
+  @Override
+  public ExecuteProceduresResponse executeProcedures(RpcController controller,
+      ExecuteProceduresRequest request) throws ServiceException {
+    return null;
+  }
+
+  @Override
+  public MergeRegionsResponse mergeRegions(RpcController controller, MergeRegionsRequest request)
+      throws ServiceException {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index 78b75d5..95c0615 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MasterTests;


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index b4bd84d..9b2ebe7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -9414,53 +9414,70 @@ public final class AdminProtos {
 
   }
 
-  public interface CloseRegionForSplitOrMergeRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeRequest)
+  public interface FlushRegionRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> 
-        getRegionList();
+    boolean hasRegion();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    int getRegionCount();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder();
+
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>optional uint64 if_older_than_ts = 2;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-        getRegionOrBuilderList();
+    boolean hasIfOlderThanTs();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>optional uint64 if_older_than_ts = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-        int index);
+    long getIfOlderThanTs();
+
+    /**
+     * <pre>
+     * whether to write a marker to WAL even if not flushed
+     * </pre>
+     *
+     * <code>optional bool write_flush_wal_marker = 3;</code>
+     */
+    boolean hasWriteFlushWalMarker();
+    /**
+     * <pre>
+     * whether to write a marker to WAL even if not flushed
+     * </pre>
+     *
+     * <code>optional bool write_flush_wal_marker = 3;</code>
+     */
+    boolean getWriteFlushWalMarker();
   }
   /**
    * <pre>
    **
-   * Closes the specified region(s) for
-   * split or merge
+   * Flushes the MemStore of the specified region.
+   * &lt;p&gt;
+   * This method is synchronous.
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
+   * Protobuf type {@code hbase.pb.FlushRegionRequest}
    */
-  public  static final class CloseRegionForSplitOrMergeRequest extends
+  public  static final class FlushRegionRequest extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitOrMergeRequest)
-      CloseRegionForSplitOrMergeRequestOrBuilder {
-    // Use CloseRegionForSplitOrMergeRequest.newBuilder() to construct.
-    private CloseRegionForSplitOrMergeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionRequest)
+      FlushRegionRequestOrBuilder {
+    // Use FlushRegionRequest.newBuilder() to construct.
+    private FlushRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private CloseRegionForSplitOrMergeRequest() {
-      region_ = java.util.Collections.emptyList();
+    private FlushRegionRequest() {
+      ifOlderThanTs_ = 0L;
+      writeFlushWalMarker_ = false;
     }
 
     @java.lang.Override
@@ -9468,7 +9485,7 @@ public final class AdminProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private CloseRegionForSplitOrMergeRequest(
+    private FlushRegionRequest(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -9492,12 +9509,26 @@ public final class AdminProtos {
               break;
             }
             case 10: {
-              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
-                mutable_bitField0_ |= 0x00000001;
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = region_.toBuilder();
+              }
+              region_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(region_);
+                region_ = subBuilder.buildPartial();
               }
-              region_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              ifOlderThanTs_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              writeFlushWalMarker_ = input.readBool();
               break;
             }
           }
@@ -9508,58 +9539,80 @@ public final class AdminProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-          region_ = java.util.Collections.unmodifiableList(region_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.Builder.class);
     }
 
+    private int bitField0_;
     public static final int REGION_FIELD_NUMBER = 1;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_;
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
-      return region_;
+    public boolean hasRegion() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-        getRegionOrBuilderList() {
-      return region_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() {
+      return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public int getRegionCount() {
-      return region_.size();
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() {
+      return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
     }
+
+    public static final int IF_OLDER_THAN_TS_FIELD_NUMBER = 2;
+    private long ifOlderThanTs_;
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>optional uint64 if_older_than_ts = 2;</code>
      */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
-      return region_.get(index);
+    public boolean hasIfOlderThanTs() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>optional uint64 if_older_than_ts = 2;</code>
      */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-        int index) {
-      return region_.get(index);
+    public long getIfOlderThanTs() {
+      return ifOlderThanTs_;
+    }
+
+    public static final int WRITE_FLUSH_WAL_MARKER_FIELD_NUMBER = 3;
+    private boolean writeFlushWalMarker_;
+    /**
+     * <pre>
+     * whether to write a marker to WAL even if not flushed
+     * </pre>
+     *
+     * <code>optional bool write_flush_wal_marker = 3;</code>
+     */
+    public boolean hasWriteFlushWalMarker() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <pre>
+     * whether to write a marker to WAL even if not flushed
+     * </pre>
+     *
+     * <code>optional bool write_flush_wal_marker = 3;</code>
+     */
+    public boolean getWriteFlushWalMarker() {
+      return writeFlushWalMarker_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -9568,11 +9621,13 @@ public final class AdminProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      for (int i = 0; i < getRegionCount(); i++) {
-        if (!getRegion(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+      if (!hasRegion()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegion().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
       memoizedIsInitialized = 1;
       return true;
@@ -9580,8 +9635,14 @@ public final class AdminProtos {
 
     public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
-      for (int i = 0; i < region_.size(); i++) {
-        output.writeMessage(1, region_.get(i));
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getRegion());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, ifOlderThanTs_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, writeFlushWalMarker_);
       }
       unknownFields.writeTo(output);
     }
@@ -9591,9 +9652,17 @@ public final class AdminProtos {
       if (size != -1) return size;
 
       size = 0;
-      for (int i = 0; i < region_.size(); i++) {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getRegion());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, ifOlderThanTs_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, region_.get(i));
+          .computeBoolSize(3, writeFlushWalMarker_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -9606,14 +9675,27 @@ public final class AdminProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) obj;
 
       boolean result = true;
-      result = result && getRegionList()
-          .equals(other.getRegionList());
+      result = result && (hasRegion() == other.hasRegion());
+      if (hasRegion()) {
+        result = result && getRegion()
+            .equals(other.getRegion());
+      }
+      result = result && (hasIfOlderThanTs() == other.hasIfOlderThanTs());
+      if (hasIfOlderThanTs()) {
+        result = result && (getIfOlderThanTs()
+            == other.getIfOlderThanTs());
+      }
+      result = result && (hasWriteFlushWalMarker() == other.hasWriteFlushWalMarker());
+      if (hasWriteFlushWalMarker()) {
+        result = result && (getWriteFlushWalMarker()
+            == other.getWriteFlushWalMarker());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -9625,67 +9707,77 @@ public final class AdminProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (getRegionCount() > 0) {
+      if (hasRegion()) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionList().hashCode();
+        hash = (53 * hash) + getRegion().hashCode();
+      }
+      if (hasIfOlderThanTs()) {
+        hash = (37 * hash) + IF_OLDER_THAN_TS_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getIfOlderThanTs());
+      }
+      if (hasWriteFlushWalMarker()) {
+        hash = (37 * hash) + WRITE_FLUSH_WAL_MARKER_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getWriteFlushWalMarker());
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(byte[] data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         byte[] data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -9697,7 +9789,7 @@ public final class AdminProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -9714,29 +9806,30 @@ public final class AdminProtos {
     /**
      * <pre>
      **
-     * Closes the specified region(s) for
-     * split or merge
+     * Flushes the MemStore of the specified region.
+     * &lt;p&gt;
+     * This method is synchronous.
      * </pre>
      *
-     * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
+     * Protobuf type {@code hbase.pb.FlushRegionRequest}
      */
     public static final class Builder extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeRequest)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequestOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.FlushRegionRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequestOrBuilder {
       public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
       }
 
       protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -9755,43 +9848,56 @@ public final class AdminProtos {
       public Builder clear() {
         super.clear();
         if (regionBuilder_ == null) {
-          region_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000001);
+          region_ = null;
         } else {
           regionBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        ifOlderThanTs_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        writeFlushWalMarker_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
       public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest(this);
         int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
         if (regionBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001)) {
-            region_ = java.util.Collections.unmodifiableList(region_);
-            bitField0_ = (bitField0_ & ~0x00000001);
-          }
           result.region_ = region_;
         } else {
           result.region_ = regionBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.ifOlderThanTs_ = ifOlderThanTs_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.writeFlushWalMarker_ = writeFlushWalMarker_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -9823,41 +9929,24 @@ public final class AdminProtos {
         return (Builder) super.addRepeatedField(field, value);
       }
       public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance()) return this;
-        if (regionBuilder_ == null) {
-          if (!other.region_.isEmpty()) {
-            if (region_.isEmpty()) {
-              region_ = other.region_;
-              bitField0_ = (bitField0_ & ~0x00000001);
-            } else {
-              ensureRegionIsMutable();
-              region_.addAll(other.region_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.region_.isEmpty()) {
-            if (regionBuilder_.isEmpty()) {
-              regionBuilder_.dispose();
-              regionBuilder_ = null;
-              region_ = other.region_;
-              bitField0_ = (bitField0_ & ~0x00000001);
-              regionBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getRegionFieldBuilder() : null;
-            } else {
-              regionBuilder_.addAllMessages(other.region_);
-            }
-          }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.getDefaultInstance()) return this;
+        if (other.hasRegion()) {
+          mergeRegion(other.getRegion());
+        }
+        if (other.hasIfOlderThanTs()) {
+          setIfOlderThanTs(other.getIfOlderThanTs());
+        }
+        if (other.hasWriteFlushWalMarker()) {
+          setWriteFlushWalMarker(other.getWriteFlushWalMarker());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -9865,10 +9954,11 @@ public final class AdminProtos {
       }
 
       public final boolean isInitialized() {
-        for (int i = 0; i < getRegionCount(); i++) {
-          if (!getRegion(i).isInitialized()) {
-            return false;
-          }
+        if (!hasRegion()) {
+          return false;
+        }
+        if (!getRegion().isInitialized()) {
+          return false;
         }
         return true;
       }
@@ -9877,11 +9967,11 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -9892,244 +9982,202 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_ =
-        java.util.Collections.emptyList();
-      private void ensureRegionIsMutable() {
-        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>(region_);
-          bitField0_ |= 0x00000001;
-         }
-      }
-
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
-        if (regionBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(region_);
-        } else {
-          return regionBuilder_.getMessageList();
-        }
-      }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public int getRegionCount() {
-        if (regionBuilder_ == null) {
-          return region_.size();
-        } else {
-          return regionBuilder_.getCount();
-        }
+      public boolean hasRegion() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() {
         if (regionBuilder_ == null) {
-          return region_.get(index);
+          return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
         } else {
-          return regionBuilder_.getMessage(index);
+          return regionBuilder_.getMessage();
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+      public Builder setRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
         if (regionBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureRegionIsMutable();
-          region_.set(index, value);
+          region_ = value;
           onChanged();
         } else {
-          regionBuilder_.setMessage(index, value);
+          regionBuilder_.setMessage(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
       public Builder setRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
         if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.set(index, builderForValue.build());
+          region_ = builderForValue.build();
           onChanged();
         } else {
-          regionBuilder_.setMessage(index, builderForValue.build());
+          regionBuilder_.setMessage(builderForValue.build());
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+      public Builder mergeRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
         if (regionBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              region_ != null &&
+              region_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) {
+            region_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial();
+          } else {
+            region_ = value;
           }
-          ensureRegionIsMutable();
-          region_.add(value);
           onChanged();
         } else {
-          regionBuilder_.addMessage(value);
+          regionBuilder_.mergeFrom(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder addRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+      public Builder clearRegion() {
         if (regionBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureRegionIsMutable();
-          region_.add(index, value);
+          region_ = null;
           onChanged();
         } else {
-          regionBuilder_.addMessage(index, value);
+          regionBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder addRegion(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.add(builderForValue.build());
-          onChanged();
-        } else {
-          regionBuilder_.addMessage(builderForValue.build());
-        }
-        return this;
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegionFieldBuilder().getBuilder();
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder addRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.add(index, builderForValue.build());
-          onChanged();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() {
+        if (regionBuilder_ != null) {
+          return regionBuilder_.getMessageOrBuilder();
         } else {
-          regionBuilder_.addMessage(index, builderForValue.build());
+          return region_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
         }
-        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder addAllRegion(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> values) {
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
+          getRegionFieldBuilder() {
         if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
-              values, region_);
-          onChanged();
-        } else {
-          regionBuilder_.addAllMessages(values);
+          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
+                  getRegion(),
+                  getParentForChildren(),
+                  isClean());
+          region_ = null;
         }
-        return this;
+        return regionBuilder_;
       }
+
+      private long ifOlderThanTs_ ;
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>optional uint64 if_older_than_ts = 2;</code>
        */
-      public Builder clearRegion() {
-        if (regionBuilder_ == null) {
-          region_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000001);
-          onChanged();
-        } else {
-          regionBuilder_.clear();
-        }
-        return this;
+      public boolean hasIfOlderThanTs() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>optional uint64 if_older_than_ts = 2;</code>
        */
-      public Builder removeRegion(int index) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.remove(index);
-          onChanged();
-        } else {
-          regionBuilder_.remove(index);
-        }
-        return this;
+      public long getIfOlderThanTs() {
+        return ifOlderThanTs_;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>optional uint64 if_older_than_ts = 2;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder(
-          int index) {
-        return getRegionFieldBuilder().getBuilder(index);
+      public Builder setIfOlderThanTs(long value) {
+        bitField0_ |= 0x00000002;
+        ifOlderThanTs_ = value;
+        onChanged();
+        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>optional uint64 if_older_than_ts = 2;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-          int index) {
-        if (regionBuilder_ == null) {
-          return region_.get(index);  } else {
-          return regionBuilder_.getMessageOrBuilder(index);
-        }
+      public Builder clearIfOlderThanTs() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        ifOlderThanTs_ = 0L;
+        onChanged();
+        return this;
       }
+
+      private boolean writeFlushWalMarker_ ;
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <pre>
+       * whether to write a marker to WAL even if not flushed
+       * </pre>
+       *
+       * <code>optional bool write_flush_wal_marker = 3;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-           getRegionOrBuilderList() {
-        if (regionBuilder_ != null) {
-          return regionBuilder_.getMessageOrBuilderList();
-        } else {
-          return java.util.Collections.unmodifiableList(region_);
-        }
+      public boolean hasWriteFlushWalMarker() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <pre>
+       * whether to write a marker to WAL even if not flushed
+       * </pre>
+       *
+       * <code>optional bool write_flush_wal_marker = 3;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder() {
-        return getRegionFieldBuilder().addBuilder(
-            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+      public boolean getWriteFlushWalMarker() {
+        return writeFlushWalMarker_;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <pre>
+       * whether to write a marker to WAL even if not flushed
+       * </pre>
+       *
+       * <code>optional bool write_flush_wal_marker = 3;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder(
-          int index) {
-        return getRegionFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+      public Builder setWriteFlushWalMarker(boolean value) {
+        bitField0_ |= 0x00000004;
+        writeFlushWalMarker_ = value;
+        onChanged();
+        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <pre>
+       * whether to write a marker to WAL even if not flushed
+       * </pre>
+       *
+       * <code>optional bool write_flush_wal_marker = 3;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder> 
-           getRegionBuilderList() {
-        return getRegionFieldBuilder().getBuilderList();
-      }
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-          getRegionFieldBuilder() {
-        if (regionBuilder_ == null) {
-          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
-                  region_,
-                  ((bitField0_ & 0x00000001) == 0x00000001),
-                  getParentForChildren(),
-                  isClean());
-          region_ = null;
-        }
-        return regionBuilder_;
+      public Builder clearWriteFlushWalMarker() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        writeFlushWalMarker_ = false;
+        onChanged();
+        return this;
       }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
@@ -10142,70 +10190,90 @@ public final class AdminProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.FlushRegionRequest)
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest DEFAULT_INSTANCE;
+    // @@protoc_insertion_point(class_scope:hbase.pb.FlushRegionRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest();
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest();
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstance() {
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitOrMergeRequest>() {
-      public CloseRegionForSplitOrMergeRequest parsePartialFrom(
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<FlushRegionRequest>() {
+      public FlushRegionRequest parsePartialFrom(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new CloseRegionForSplitOrMergeRequest(input, extensionRegistry);
+          return new FlushRegionRequest(input, extensionRegistry);
       }
     };
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionRequest> parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionRequest> getParserForType() {
       return PARSER;
     }
 
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstanceForType() {
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface CloseRegionForSplitOrMergeResponseOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeResponse)
+  public interface FlushRegionResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionResponse)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>required bool closed = 1;</code>
+     * <code>required uint64 last_flush_time = 1;</code>
      */
-    boolean hasClosed();
+    boolean hasLastFlushTime();
     /**
-     * <code>required bool closed = 1;</code>
+     * <code>required uint64 last_flush_time = 1;</code>
      */
-    boolean getClosed();
+    long getLastFlushTime();
+
+    /**
+     * <code>optional bool flushed = 2;</code>
+     */
+    boolean hasFlushed();
+    /**
+     * <code>optional bool flushed = 2;</code>
+     */
+    boolean getFlushed();
+
+    /**
+     * <code>optional bool wrote_flush_wal_marker = 3;</code>
+     */
+    boolean hasWroteFlushWalMarker();
+    /**
+     * <code>optional bool wrote_flush_wal_marker = 3;</code>
+     */
+    boolean getWroteFlushWalMarker();
   }
   /**
-   * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
+   * Protobuf type {@code hbase.pb.FlushRegionResponse}
    */
-  public  static final class CloseRegionForSplitOrMergeResponse extends
+  public  static final class FlushRegionResponse extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.CloseRegionForSplitOrMergeResponse)
-      CloseRegionForSplitOrMergeResponseOrBuilder {
-    // Use CloseRegionForSplitOrMergeResponse.newBuilder() to construct.
-    private CloseRegionForSplitOrMergeResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionResponse)
+      FlushRegionResponseOrBuilder {
+    // Use FlushRegionResponse.newBuilder() to construct.
+    private FlushRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private CloseRegionForSplitOrMergeResponse() {
-      closed_ = false;
+    private FlushRegionResponse() {
+      lastFlushTime_ = 0L;
+      flushed_ = false;
+      wroteFlushWalMarker_ = false;
     }
 
     @java.lang.Override
@@ -10213,7 +10281,7 @@ public final class AdminProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private CloseRegionForSplitOrMergeResponse(
+    private FlushRegionResponse(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -10238,7 +10306,17 @@ public final class AdminProtos {
             }
             case 8: {
               bitField0_ |= 0x00000001;
-              closed_ = input.readBool();
+              lastFlushTime_ = input.readUInt64();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              flushed_ = input.readBool();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              wroteFlushWalMarker_ = input.readBool();
               break;
             }
           }
@@ -10255,30 +10333,60 @@ public final class AdminProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.Builder.class);
     }
 
     private int bitField0_;
-    public static final int CLOSED_FIELD_NUMBER = 1;
-    private boolean closed_;
+    public static final int LAST_FLUSH_TIME_FIELD_NUMBER = 1;
+    private long lastFlushTime_;
     /**
-     * <code>required bool closed = 1;</code>
+     * <code>required uint64 last_flush_time = 1;</code>
      */
-    public boolean hasClosed() {
+    public boolean hasLastFlushTime() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required bool closed = 1;</code>
+     * <code>required uint64 last_flush_time = 1;</code>
      */
-    public boolean getClosed() {
-      return closed_;
+    public long getLastFlushTime() {
+      return lastFlushTime_;
+    }
+
+    public static final int FLUSHED_FIELD_NUMBER = 2;
+    private boolean flushed_;
+    /**
+     * <code>optional bool flushed = 2;</code>
+     */
+    public boolean hasFlushed() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool flushed = 2;</code>
+     */
+    public boolean getFlushed() {
+      return flushed_;
+    }
+
+    public static final int WROTE_FLUSH_WAL_MARKER_FIELD_NUMBER = 3;
+    private boolean wroteFlushWalMarker_;
+    /**
+     * <code>optional bool wrote_flush_wal_marker = 3;</code>
+     */
+    public boolean hasWroteFlushWalMarker() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bool wrote_flush_wal_marker = 3;</code>
+     */
+    public boolean getWroteFlushWalMarker() {
+      return wroteFlushWalMarker_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -10287,7 +10395,7 @@ public final class AdminProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      if (!hasClosed()) {
+      if (!hasLastFlushTime()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -10298,7 +10406,13 @@ public final class AdminProtos {
     public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, closed_);
+        output.writeUInt64(1, lastFlushTime_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, flushed_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, wroteFlushWalMarker_);
       }
       unknownFields.writeTo(output);
     }
@@ -10310,7 +10424,15 @@ public final class AdminProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, closed_);
+          .computeUInt64Size(1, lastFlushTime_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, flushed_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, wroteFlushWalMarker_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -10323,16 +10445,26 @@ public final class AdminProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) obj;
 
       boolean result = true;
-      result = result && (hasClosed() == other.hasClosed());
-      if (hasClosed()) {
-        result = result && (getClosed()
-            == other.getClosed());
+      result = result && (hasLastFlushTime() == other.hasLastFlushTime());
+      if (hasLastFlushTime()) {
+        result = result && (getLastFlushTime()
+            == other.getLastFlushTime());
+      }
+      result = result && (hasFlushed() == other.hasFlushed());
+      if (hasFlushed()) {
+        result = result && (getFlushed()
+            == other.getFlushed());
+      }
+      result = result && (hasWroteFlushWalMarker() == other.hasWroteFlushWalMarker());
+      if (hasWroteFlushWalMarker()) {
+        result = result && (getWroteFlushWalMarker()
+            == other.getWroteFlushWalMarker());
       }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
@@ -10345,68 +10477,78 @@ public final class AdminProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasClosed()) {
-        hash = (37 * hash) + CLOSED_FIELD_NUMBER;
+      if (hasLastFlushTime()) {
+        hash = (37 * hash) + LAST_FLUSH_TIME_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getLastFlushTime());
+      }
+      if (hasFlushed()) {
+        hash = (37 * hash) + FLUSHED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
-            getClosed());
+            getFlushed());
+      }
+      if (hasWroteFlushWalMarker()) {
+        hash = (37 * hash) + WROTE_FLUSH_WAL_MARKER_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getWroteFlushWalMarker());
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(byte[] data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         byte[] data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -10418,7 +10560,7 @@ public final class AdminProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -10433,25 +10575,25 @@ public final class AdminProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
+     * Protobuf type {@code hbase.pb.FlushRegionResponse}
      */
     public static final class Builder extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeResponse)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponseOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.FlushRegionResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponseOrBuilder {
       public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
       }
 
       protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -10468,36 +10610,48 @@ public final class AdminProtos {
       }
       public Builder clear() {
         super.clear();
-        closed_ = false;
+        lastFlushTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000001);
+        flushed_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        wroteFlushWalMarker_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
       public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.closed_ = closed_;
+        result.lastFlushTime_ = lastFlushTime_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.flushed_ = flushed_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.wroteFlushWalMarker_ = wroteFlushWalMarker_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -10530,18 +10684,24 @@ public final class AdminProtos {
         return (Builder) super.addRepeatedField(field, value);
       }
       public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance()) return this;
-        if (other.hasClosed()) {
-          setClosed(other.getClosed());
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.getDefaultInstance()) return this;
+        if (other.hasLastFlushTime()) {
+          setLastFlushTime(other.getLastFlushTime());
+        }
+        if (other.hasFlushed()) {
+          setFlushed(other.getFlushed());
+        }
+        if (other.hasWroteFlushWalMarker()) {
+          setWroteFlushWalMarker(other.getWroteFlushWalMarker());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -10549,7 +10709,7 @@ public final class AdminProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasClosed()) {
+        if (!hasLastFlushTime()) {
           return false;
         }
         return true;
@@ -10559,11 +10719,11 @@ public final class AdminProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -10574,34 +10734,98 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private boolean closed_ ;
+      private long lastFlushTime_ ;
       /**
-       * <code>required bool closed = 1;</code>
+       * <code>required uint64 last_flush_time = 1;</code>
        */
-      public boolean hasClosed() {
+      public boolean hasLastFlushTime() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required bool closed = 1;</code>
+       * <code>required uint64 last_flush_time = 1;</code>
        */
-      public boolean getClosed() {
-        return closed_;
+      public long getLastFlushTime() {
+        return lastFlushTime_;
       }
       /**
-       * <code>required bool closed = 1;</code>
+       * <code>required uint64 last_flush_time = 1;</code>
        */
-      public Builder setClosed(boolean value) {
+      public Builder setLastFlushTime(long value) {
         bitField0_ |= 0x00000001;
-        closed_ = value;
+        lastFlushTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required bool closed = 1;</code>
+       * <code>required uint64 last_flush_time = 1;</code>
        */
-      public Builder clearClosed() {
+      public Builder clearLastFlushTime() {
         bitField0_ = (bitField0_ & ~0x00000001);
-        closed_ = false;
+        lastFlushTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      private boolean flushed_ ;
+      /**
+       * <code>optional bool flushed = 2;</code>
+       */
+      public boolean hasFlushed() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool flushed = 2;</code>
+       */
+      public boolean getFlushed() {
+        return flushed_;
+      }
+      /**
+       * <code>optional bool flushed = 2;</code>
+       */
+      public Builder setFlushed(boolean value) {
+        bitField0_ |= 0x00000002;
+        flushed_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool flushed = 2;</code>
+       */
+      public Builder clearFlushed() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        flushed_ = false;
+        onChanged();
+        return this;
+      }
+
+      private boolean wroteFlushWalMarker_ ;
+      /**
+       * <code>optional bool wrote_flush_wal_marker = 3;</code>
+       */
+      public boolean hasWroteFlushWalMarker() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional bool wrote_flush_wal_marker = 3;</code>
+       */
+      public boolean getWroteFlushWalMarker() {
+        return wroteFlushWalMarker_;
+      }
+      /**
+       * <code>optional bool wrote_flush_wal_marker = 3;</code>
+       */
+      public Builder setWroteFlushWalMarker(boolean value) {
+        bitField0_ |= 0x00000004;
+        wroteFlushWalMarker_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool wrote_flush_wal_marker = 3;</code>
+       */
+      public Builder clearWroteFlushWalMarker() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        wroteFlushWalMarker_ = false;
         onChanged();
         return this;
       }
@@ -10616,46 +10840,46 @@ public final class AdminProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitOrMergeResponse)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.FlushRegionResponse)
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitOrMergeResponse)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse DEFAULT_INSTANCE;
+    // @@protoc_insertion_point(class_scope:hbase.pb.FlushRegionResponse)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse();
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse();
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstance() {
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<CloseRegionForSplitOrMergeResponse>() {
-      public CloseRegionForSplitOrMergeResponse parsePartialFrom(
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionResponse>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<FlushRegionResponse>() {
+      public FlushRegionResponse parsePartialFrom(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new CloseRegionForSplitOrMergeResponse(input, extensionRegistry);
+          return new FlushRegionResponse(input, extensionRegistry);
       }
     };
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse> parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionResponse> parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeResponse> getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionResponse> getParserForType() {
       return PARSER;
     }
 
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse getDefaultInstanceForType() {
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface FlushRegionRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionRequest)
+  public interface SplitRegionRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.SplitRegionRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
@@ -10672,52 +10896,36 @@ public final class AdminProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder();
 
     /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
-     */
-    boolean hasIfOlderThanTs();
-    /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
-     */
-    long getIfOlderThanTs();
-
-    /**
-     * <pre>
-     * whether to write a marker to WAL even if not flushed
-     * </pre>
-     *
-     * <code>optional bool write_flush_wal_marker = 3;</code>
+     * <code>optional bytes split_point = 2;</code>
      */
-    boolean hasWriteFlushWalMarker();
+    boolean hasSplitPoint();
     /**
-     * <pre>
-     * whether to write a marker to WAL even if not flushed
-     * </pre>
-     *
-     * <code>optional bool write_flush_wal_marker = 3;</code>
+     * <code>optional bytes split_point = 2;</code>
      */
-    boolean getWriteFlushWalMarker();
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitPoint();
   }
   /**
    * <pre>
    **
-   * Flushes the MemStore of the specified region.
+   * Splits the specified region.
    * &lt;p&gt;
-   * This method is synchronous.
+   * This method currently flushes the region and then forces a compaction which
+   * will then trigger a split.  The flush is done synchronously but the
+   * compaction is asynchronous.
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.FlushRegionRequest}
+   * Protobuf type {@code hbase.pb.SplitRegionRequest}
    */
-  public  static final class FlushRegionRequest extends
+  public  static final class SplitRegionRequest extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.FlushRegionRequest)
-      FlushRegionRequestOrBuilder {
-    // Use FlushRegionRequest.newBuilder() to construct.
-    private FlushRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.SplitRegionRequest)
+      SplitRegionRequestOrBuilder {
+    // Use SplitRegionRequest.newBuilder() to construct.
+    private SplitRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private FlushRegionRequest() {
-      ifOlderThanTs_ = 0L;
-      writeFlushWalMarker_ = false;
+    private SplitRegionRequest() {
+      splitPoint_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
     }
 
     @java.lang.Override
@@ -10725,7 +10933,7 @@ public final class AdminProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private FlushRegionRequest(
+    private SplitRegionRequest(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -10761,14 +10969,9 @@ public final class AdminProtos {
               bitField0_ |= 0x00000001;
               break;
             }
-            case 16: {
+            case 18: {
               bitField0_ |= 0x00000002;
-              ifOlderThanTs_ = input.readUInt64();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              writeFlushWalMarker_ = input.readBool();
+              splitPoint_ = input.readBytes();
               break;
             }
           }
@@ -10785,14 +10988,14 @@ public final class AdminProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_FlushRegionRequest_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_SplitRegionRequest_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.p

<TRUNCATED>

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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
new file mode 100644
index 0000000..aa1ed00
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -0,0 +1,717 @@
+/**
+ * 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.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collection;
+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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaMutationAnnotation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.exceptions.MergeRegionException;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.CatalogJanitor;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+/**
+ * The procedure to Merge a region in a table.
+ */
+@InterfaceAudience.Private
+public class MergeTableRegionsProcedure
+    extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
+  private static final Log LOG = LogFactory.getLog(MergeTableRegionsProcedure.class);
+
+  private Boolean traceEnabled;
+
+  private ServerName regionLocation;
+  private String regionsToMergeListFullName;
+
+  private HRegionInfo[] regionsToMerge;
+  private HRegionInfo mergedRegion;
+  private boolean forcible;
+
+  public MergeTableRegionsProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+      final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB) throws IOException {
+    this(env, regionToMergeA, regionToMergeB, false);
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+      final HRegionInfo regionToMergeA, final HRegionInfo regionToMergeB,
+      final boolean forcible) throws MergeRegionException {
+    this(env, new HRegionInfo[] {regionToMergeA, regionToMergeB}, forcible);
+  }
+
+  public MergeTableRegionsProcedure(final MasterProcedureEnv env,
+      final HRegionInfo[] regionsToMerge, final boolean forcible)
+      throws MergeRegionException {
+    super(env);
+
+    // Check daughter regions and make sure that we have valid daughter regions
+    // before doing the real work.
+    checkRegionsToMerge(regionsToMerge, forcible);
+
+    // WARN: make sure there is no parent region of the two merging regions in
+    // hbase:meta If exists, fixing up daughters would cause daughter regions(we
+    // have merged one) online again when we restart master, so we should clear
+    // the parent region to prevent the above case
+    // Since HBASE-7721, we don't need fix up daughters any more. so here do nothing
+    this.regionsToMerge = regionsToMerge;
+    this.mergedRegion = createMergedRegionInfo(regionsToMerge);
+    this.forcible = forcible;
+
+    this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
+  }
+
+  private static void checkRegionsToMerge(final HRegionInfo[] regionsToMerge,
+      final boolean forcible) throws MergeRegionException {
+    // For now, we only merge 2 regions.
+    // It could be extended to more than 2 regions in the future.
+    if (regionsToMerge == null || regionsToMerge.length != 2) {
+      throw new MergeRegionException("Expected to merge 2 regions, got: " +
+        Arrays.toString(regionsToMerge));
+    }
+
+    checkRegionsToMerge(regionsToMerge[0], regionsToMerge[1], forcible);
+  }
+
+  private static void checkRegionsToMerge(final HRegionInfo regionToMergeA,
+      final HRegionInfo regionToMergeB, final boolean forcible) throws MergeRegionException {
+    if (!regionToMergeA.getTable().equals(regionToMergeB.getTable())) {
+      throw new MergeRegionException("Can't merge regions from two different tables: " +
+        regionToMergeA + ", " + regionToMergeB);
+    }
+
+    if (regionToMergeA.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
+        regionToMergeB.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
+      throw new MergeRegionException("Can't merge non-default replicas");
+    }
+
+    if (!HRegionInfo.areAdjacent(regionToMergeA, regionToMergeB)) {
+      String msg = "Unable to merge not adjacent regions " + regionToMergeA.getShortNameToLog() +
+          ", " + regionToMergeB.getShortNameToLog() + " where forcible = " + forcible;
+      LOG.warn(msg);
+      if (!forcible) {
+        throw new MergeRegionException(msg);
+      }
+    }
+  }
+
+  private static HRegionInfo createMergedRegionInfo(final HRegionInfo[] regionsToMerge) {
+    return createMergedRegionInfo(regionsToMerge[0], regionsToMerge[1]);
+  }
+
+  /**
+   * Create merged region info through the specified two regions
+   */
+  private static HRegionInfo createMergedRegionInfo(final HRegionInfo regionToMergeA,
+      final HRegionInfo regionToMergeB) {
+    // Choose the smaller as start key
+    final byte[] startKey;
+    if (regionToMergeA.compareTo(regionToMergeB) <= 0) {
+      startKey = regionToMergeA.getStartKey();
+    } else {
+      startKey = regionToMergeB.getStartKey();
+    }
+
+    // Choose the bigger as end key
+    final byte[] endKey;
+    if (Bytes.equals(regionToMergeA.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
+        || (!Bytes.equals(regionToMergeB.getEndKey(), HConstants.EMPTY_BYTE_ARRAY)
+            && Bytes.compareTo(regionToMergeA.getEndKey(), regionToMergeB.getEndKey()) > 0)) {
+      endKey = regionToMergeA.getEndKey();
+    } else {
+      endKey = regionToMergeB.getEndKey();
+    }
+
+    // Merged region is sorted between two merging regions in META
+    final long rid = getMergedRegionIdTimestamp(regionToMergeA, regionToMergeB);
+    return new HRegionInfo(regionToMergeA.getTable(), startKey, endKey, false, rid);
+  }
+
+  private static long getMergedRegionIdTimestamp(final HRegionInfo regionToMergeA,
+      final HRegionInfo regionToMergeB) {
+    long rid = EnvironmentEdgeManager.currentTime();
+    // Regionid is timestamp. Merged region's id can't be less than that of
+    // merging regions else will insert at wrong location in hbase:meta (See HBASE-710).
+    if (rid < regionToMergeA.getRegionId() || rid < regionToMergeB.getRegionId()) {
+      LOG.warn("Clock skew; merging regions id are " + regionToMergeA.getRegionId()
+          + " and " + regionToMergeB.getRegionId() + ", but current time here is " + rid);
+      rid = Math.max(regionToMergeA.getRegionId(), regionToMergeB.getRegionId()) + 1;
+    }
+    return rid;
+  }
+
+  @Override
+  protected Flow executeFromState(
+      final MasterProcedureEnv env,
+      final MergeTableRegionsState state) throws InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    try {
+      switch (state) {
+      case MERGE_TABLE_REGIONS_PREPARE:
+        prepareMergeRegion(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION);
+        break;
+      case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
+        preMergeRegions(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE);
+        break;
+      case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
+        setRegionStateToMerging(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CLOSE_REGIONS);
+        break;
+      case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
+        addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
+        break;
+      case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
+        createMergedRegion(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION);
+        break;
+      case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
+        preMergeRegionsCommit(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_UPDATE_META);
+        break;
+      case MERGE_TABLE_REGIONS_UPDATE_META:
+        updateMetaForMergedRegions(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION);
+        break;
+      case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+        postMergeRegionsCommit(env);
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_OPEN_MERGED_REGION);
+        break;
+      case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+        addChildProcedure(createAssignProcedures(env, getRegionReplication(env)));
+        setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
+        break;
+      case MERGE_TABLE_REGIONS_POST_OPERATION:
+        postCompletedMergeRegions(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
+        " in the table " + getTableName() + " (in state=" + state + ")", e);
+
+      setFailure("master-merge-regions", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(
+      final MasterProcedureEnv env,
+      final MergeTableRegionsState state) throws IOException, InterruptedException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case MERGE_TABLE_REGIONS_POST_OPERATION:
+      case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+      case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+      case MERGE_TABLE_REGIONS_UPDATE_META:
+        String msg = this + " We are in the " + state + " state."
+            + " It is complicated to rollback the merge operation that region server is working on."
+            + " Rollback is not supported and we should let the merge operation to complete";
+        LOG.warn(msg);
+        // PONR
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      case MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION:
+        break;
+      case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
+        cleanupMergedRegion(env);
+        break;
+      case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
+        rollbackCloseRegionsForMerge(env);
+        break;
+      case MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE:
+        setRegionStateToRevertMerging(env);
+        break;
+      case MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION:
+        postRollBackMergeRegions(env);
+        break;
+      case MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS:
+        break; // nothing to rollback
+      case MERGE_TABLE_REGIONS_PREPARE:
+        break; // nothing to rollback
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (Exception e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
+          + getRegionsToMergeListFullNameString() + " in table " + getTableName(), e);
+      throw e;
+    }
+  }
+
+  /*
+   * Check whether we are in the state that can be rollback
+   */
+  @Override
+  protected boolean isRollbackSupported(final MergeTableRegionsState state) {
+    switch (state) {
+    case MERGE_TABLE_REGIONS_POST_OPERATION:
+    case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
+    case MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION:
+    case MERGE_TABLE_REGIONS_UPDATE_META:
+        // It is not safe to rollback if we reach to these states.
+        return false;
+      default:
+        break;
+    }
+    return true;
+  }
+
+  @Override
+  protected MergeTableRegionsState getState(final int stateId) {
+    return MergeTableRegionsState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final MergeTableRegionsState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected MergeTableRegionsState getInitialState() {
+    return MergeTableRegionsState.MERGE_TABLE_REGIONS_PREPARE;
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    final MasterProcedureProtos.MergeTableRegionsStateData.Builder mergeTableRegionsMsg =
+        MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
+        .setMergedRegionInfo(HRegionInfo.convert(mergedRegion))
+        .setForcible(forcible);
+    for (int i = 0; i < regionsToMerge.length; ++i) {
+      mergeTableRegionsMsg.addRegionInfo(HRegionInfo.convert(regionsToMerge[i]));
+    }
+    mergeTableRegionsMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    final MasterProcedureProtos.MergeTableRegionsStateData mergeTableRegionsMsg =
+        MasterProcedureProtos.MergeTableRegionsStateData.parseDelimitedFrom(stream);
+    setUser(MasterProcedureUtil.toUserInfo(mergeTableRegionsMsg.getUserInfo()));
+
+    assert(mergeTableRegionsMsg.getRegionInfoCount() == 2);
+    regionsToMerge = new HRegionInfo[mergeTableRegionsMsg.getRegionInfoCount()];
+    for (int i = 0; i < regionsToMerge.length; i++) {
+      regionsToMerge[i] = HRegionInfo.convert(mergeTableRegionsMsg.getRegionInfo(i));
+    }
+
+    mergedRegion = HRegionInfo.convert(mergeTableRegionsMsg.getMergedRegionInfo());
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append("(table=");
+    sb.append(getTableName());
+    sb.append(" regions=");
+    sb.append(getRegionsToMergeListFullNameString());
+    sb.append(" forcible=");
+    sb.append(forcible);
+    sb.append(")");
+  }
+
+  @Override
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
+
+    if (env.getProcedureScheduler().waitRegions(this, getTableName(),
+        mergedRegion, regionsToMerge[0], regionsToMerge[1])) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakeRegions(this, getTableName(),
+      mergedRegion, regionsToMerge[0], regionsToMerge[1]);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return mergedRegion.getTable();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.MERGE;
+  }
+
+  /**
+   * Prepare merge and do some check
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareMergeRegion(final MasterProcedureEnv env) throws IOException {
+    // Note: the following logic assumes that we only have 2 regions to merge.  In the future,
+    // if we want to extend to more than 2 regions, the code needs to modify a little bit.
+    //
+    CatalogJanitor catalogJanitor = env.getMasterServices().getCatalogJanitor();
+    boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
+    if (regionAHasMergeQualifier
+        || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
+      String msg = "Skip merging regions " + getRegionsToMergeListFullNameString()
+        + ", because region "
+        + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
+              .getEncodedName()) + " has merge qualifier";
+      LOG.warn(msg);
+      throw new MergeRegionException(msg);
+    }
+
+    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
+    RegionState regionStateA = regionStates.getRegionState(regionsToMerge[0].getEncodedName());
+    RegionState regionStateB = regionStates.getRegionState(regionsToMerge[1].getEncodedName());
+    if (regionStateA == null || regionStateB == null) {
+      throw new UnknownRegionException(
+        regionStateA == null ?
+            regionsToMerge[0].getEncodedName() : regionsToMerge[1].getEncodedName());
+    }
+
+    if (!regionStateA.isOpened() || !regionStateB.isOpened()) {
+      throw new MergeRegionException(
+        "Unable to merge regions not online " + regionStateA + ", " + regionStateB);
+    }
+  }
+
+  /**
+   * Pre merge region action
+   * @param env MasterProcedureEnv
+   **/
+  private void preMergeRegions(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
+      if (ret) {
+        throw new IOException(
+          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+      }
+    }
+  }
+
+  /**
+   * Action after rollback a merge table regions action.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void postRollBackMergeRegions(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postRollBackMergeRegionsAction(regionsToMerge, getUser());
+    }
+  }
+
+  /**
+   * Set the region states to MERGING state
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  public void setRegionStateToMerging(final MasterProcedureEnv env) throws IOException {
+    //transition.setTransitionCode(TransitionCode.READY_TO_MERGE);
+  }
+
+  /**
+   * Rollback the region state change
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void setRegionStateToRevertMerging(final MasterProcedureEnv env) throws IOException {
+    //transition.setTransitionCode(TransitionCode.MERGE_REVERTED);
+  }
+
+  /**
+   * Create merged region
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void createMergedRegion(final MasterProcedureEnv env) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+    final FileSystem fs = mfs.getFileSystem();
+    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
+    regionFs.createMergesDir();
+
+    mergeStoreFiles(env, regionFs, regionFs.getMergesDir());
+    HRegionFileSystem regionFs2 = HRegionFileSystem.openRegionFromFileSystem(
+      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[1], false);
+    mergeStoreFiles(env, regionFs2, regionFs.getMergesDir());
+
+    regionFs.commitMergedRegion(mergedRegion);
+  }
+
+  /**
+   * Create reference file(s) of merging regions under the merges directory
+   * @param env MasterProcedureEnv
+   * @param regionFs region file system
+   * @param mergedDir the temp directory of merged region
+   * @throws IOException
+   */
+  private void mergeStoreFiles(
+      final MasterProcedureEnv env, final HRegionFileSystem regionFs, final Path mergedDir)
+      throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Configuration conf = env.getMasterConfiguration();
+    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
+
+    for (String family: regionFs.getFamilies()) {
+      final HColumnDescriptor hcd = htd.getFamily(family.getBytes());
+      final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
+
+      if (storeFiles != null && storeFiles.size() > 0) {
+        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
+        for (StoreFileInfo storeFileInfo: storeFiles) {
+          // Create reference file(s) of the region in mergedDir
+          regionFs.mergeStoreFile(
+            mergedRegion,
+            family,
+            new StoreFile(
+              mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
+            mergedDir);
+        }
+      }
+    }
+  }
+
+  /**
+   * Clean up merged region
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void cleanupMergedRegion(final MasterProcedureEnv env) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), regionsToMerge[0].getTable());
+    final FileSystem fs = mfs.getFileSystem();
+    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
+      env.getMasterConfiguration(), fs, tabledir, regionsToMerge[0], false);
+    regionFs.cleanupMergedRegion(mergedRegion);
+  }
+
+  /**
+   * Rollback close regions
+   * @param env MasterProcedureEnv
+   **/
+  private void rollbackCloseRegionsForMerge(final MasterProcedureEnv env) throws IOException {
+    // Check whether the region is closed; if so, open it in the same server
+    final int regionReplication = getRegionReplication(env);
+    final ServerName serverName = getServerName(env);
+
+    final AssignProcedure[] procs =
+        new AssignProcedure[regionsToMerge.length * regionReplication];
+    int procsIdx = 0;
+    for (int i = 0; i < regionsToMerge.length; ++i) {
+      for (int j = 0; j < regionReplication; ++j) {
+        final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
+        procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
+      }
+    }
+    env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
+  }
+
+  private UnassignProcedure[] createUnassignProcedures(final MasterProcedureEnv env,
+      final int regionReplication) {
+    final UnassignProcedure[] procs =
+        new UnassignProcedure[regionsToMerge.length * regionReplication];
+    int procsIdx = 0;
+    for (int i = 0; i < regionsToMerge.length; ++i) {
+      for (int j = 0; j < regionReplication; ++j) {
+        final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
+        procs[procsIdx++] = env.getAssignmentManager().createUnassignProcedure(hri,null,true);
+      }
+    }
+    return procs;
+  }
+
+  private AssignProcedure[] createAssignProcedures(final MasterProcedureEnv env,
+      final int regionReplication) {
+    final ServerName targetServer = getServerName(env);
+    final AssignProcedure[] procs = new AssignProcedure[regionReplication];
+    for (int i = 0; i < procs.length; ++i) {
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
+      procs[i] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
+    }
+    return procs;
+  }
+
+  private int getRegionReplication(final MasterProcedureEnv env) throws IOException {
+    final HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
+    return htd.getRegionReplication();
+  }
+
+  /**
+   * Post merge region action
+   * @param env MasterProcedureEnv
+   **/
+  private void preMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      @MetaMutationAnnotation
+      final List<Mutation> metaEntries = new ArrayList<Mutation>();
+      boolean ret = cpHost.preMergeRegionsCommit(regionsToMerge, metaEntries, getUser());
+
+      if (ret) {
+        throw new IOException(
+          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+      }
+      try {
+        for (Mutation p : metaEntries) {
+          HRegionInfo.parseRegionName(p.getRow());
+        }
+      } catch (IOException e) {
+        LOG.error("Row key of mutation from coprocessor is not parsable as region name."
+          + "Mutations from coprocessor should only be for hbase:meta table.", e);
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Add merged region to META and delete original regions.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException {
+    final ServerName serverName = getServerName(env);
+    env.getAssignmentManager().markRegionAsMerged(mergedRegion, serverName,
+      regionsToMerge[0], regionsToMerge[1]);
+  }
+
+  /**
+   * Post merge region action
+   * @param env MasterProcedureEnv
+   **/
+  private void postMergeRegionsCommit(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postMergeRegionsCommit(regionsToMerge, mergedRegion, getUser());
+    }
+  }
+
+  /**
+   * Post merge region action
+   * @param env MasterProcedureEnv
+   **/
+  private void postCompletedMergeRegions(final MasterProcedureEnv env) throws IOException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postCompletedMergeRegionsAction(regionsToMerge, mergedRegion, getUser());
+    }
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param env MasterProcedureEnv
+   * @return serverName
+   */
+  private ServerName getServerName(final MasterProcedureEnv env) {
+    if (regionLocation == null) {
+      regionLocation = env.getAssignmentManager().getRegionStates()
+        .getRegionServerOfRegion(regionsToMerge[0]);
+    }
+    return regionLocation;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @param fullName whether return only encoded name
+   * @return region names in a list
+   */
+  private String getRegionsToMergeListFullNameString() {
+    if (regionsToMergeListFullName == null) {
+      final StringBuilder sb = new StringBuilder("[");
+      int i = 0;
+      while(i < regionsToMerge.length - 1) {
+        sb.append(regionsToMerge[i].getRegionNameAsString());
+        sb.append(", ");
+        i++;
+      }
+      sb.append(regionsToMerge[i].getRegionNameAsString());
+      sb.append("]");
+      regionsToMergeListFullName = sb.toString();
+    }
+    return regionsToMergeListFullName;
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
new file mode 100644
index 0000000..916ef5e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
@@ -0,0 +1,147 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.master.RegionPlan;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
+
+/**
+ * Procedure that implements a RegionPlan.
+ * It first runs an unassign subprocedure followed
+ * by an assign subprocedure.
+ */
+@InterfaceAudience.Private
+public class MoveRegionProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, MoveRegionState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(MoveRegionProcedure.class);
+
+  private RegionPlan plan;
+
+  public MoveRegionProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  public MoveRegionProcedure(final RegionPlan plan) {
+    this.plan = plan;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final MoveRegionState state)
+      throws InterruptedException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    switch (state) {
+      case MOVE_REGION_UNASSIGN:
+        addChildProcedure(new UnassignProcedure(plan.getRegionInfo(), plan.getDestination(), true));
+        setNextState(MoveRegionState.MOVE_REGION_ASSIGN);
+        break;
+      case MOVE_REGION_ASSIGN:
+        addChildProcedure(new AssignProcedure(plan.getRegionInfo(), plan.getDestination()));
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final MoveRegionState state)
+      throws IOException {
+    // no-op
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    return false;
+  }
+
+  @Override
+  public void toStringClassDetails(final StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append("(plan=");
+    sb.append(plan);
+    sb.append(")");
+  }
+
+  @Override
+  protected MoveRegionState getInitialState() {
+    return MoveRegionState.MOVE_REGION_UNASSIGN;
+  }
+
+  @Override
+  protected int getStateId(final MoveRegionState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected MoveRegionState getState(final int stateId) {
+    return MoveRegionState.valueOf(stateId);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return plan.getRegionInfo().getTable();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REGION_EDIT;
+  }
+
+  @Override
+  protected void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    final MoveRegionStateData.Builder state = MoveRegionStateData.newBuilder()
+        .setRegionInfo(HRegionInfo.convert(plan.getRegionInfo()))
+        .setSourceServer(ProtobufUtil.toServerName(plan.getSource()))
+        .setDestinationServer(ProtobufUtil.toServerName(plan.getDestination()));
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  protected void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    final MoveRegionStateData state = MoveRegionStateData.parseDelimitedFrom(stream);
+    final HRegionInfo regionInfo = HRegionInfo.convert(state.getRegionInfo());
+    final ServerName sourceServer = ProtobufUtil.toServerName(state.getSourceServer());
+    final ServerName destinationServer = ProtobufUtil.toServerName(state.getDestinationServer());
+    this.plan = new RegionPlan(regionInfo, sourceServer, destinationServer);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
new file mode 100644
index 0000000..ba33974
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -0,0 +1,327 @@
+/**
+ *
+ * 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.assignment;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.MultiHConnection;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.zookeeper.KeeperException;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Store Region State to hbase:meta table.
+ */
+@InterfaceAudience.Private
+public class RegionStateStore {
+  private static final Log LOG = LogFactory.getLog(RegionStateStore.class);
+
+  /** The delimiter for meta columns for replicaIds &gt; 0 */
+  protected static final char META_REPLICA_ID_DELIMITER = '_';
+
+  private final MasterServices master;
+
+  private MultiHConnection multiHConnection;
+
+  public RegionStateStore(final MasterServices master) {
+    this.master = master;
+  }
+
+  public void start() throws IOException {
+  }
+
+  public void stop() {
+    if (multiHConnection != null) {
+      multiHConnection.close();
+      multiHConnection = null;
+    }
+  }
+
+  public interface RegionStateVisitor {
+    void visitRegionState(HRegionInfo regionInfo, State state,
+      ServerName regionLocation, ServerName lastHost, long openSeqNum);
+  }
+
+  public void visitMeta(final RegionStateVisitor visitor) throws IOException {
+    MetaTableAccessor.fullScanRegions(master.getConnection(), new MetaTableAccessor.Visitor() {
+      final boolean isDebugEnabled = LOG.isDebugEnabled();
+
+      @Override
+      public boolean visit(final Result r) throws IOException {
+        if (r !=  null && !r.isEmpty()) {
+          long st = System.currentTimeMillis();
+          visitMetaEntry(visitor, r);
+          long et = System.currentTimeMillis();
+          LOG.info("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st));
+        } else if (isDebugEnabled) {
+          LOG.debug("null result from meta - ignoring but this is strange.");
+        }
+        return true;
+      }
+    });
+  }
+
+  private void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
+      throws IOException {
+    final RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
+    if (rl == null) return;
+
+    final HRegionLocation[] locations = rl.getRegionLocations();
+    if (locations == null) return;
+
+    for (int i = 0; i < locations.length; ++i) {
+      final HRegionLocation hrl = locations[i];
+      if (hrl == null) continue;
+
+      final HRegionInfo regionInfo = hrl.getRegionInfo();
+      if (regionInfo == null) continue;
+
+      final int replicaId = regionInfo.getReplicaId();
+      final State state = getRegionState(result, replicaId);
+
+      final ServerName lastHost = hrl.getServerName();
+      final ServerName regionLocation = getRegionServer(result, replicaId);
+      final long openSeqNum = -1;
+
+      // TODO: move under trace, now is visible for debugging
+      LOG.info(String.format("Load meta entry region=%s state=%s lastHost=%s regionLocation=%s",
+        regionInfo, state, lastHost, regionLocation));
+
+      visitor.visitRegionState(regionInfo, state, regionLocation, lastHost, openSeqNum);
+    }
+  }
+
+  public void updateRegionLocation(final HRegionInfo regionInfo, final State state,
+      final ServerName regionLocation, final ServerName lastHost, final long openSeqNum)
+      throws IOException {
+    if (regionInfo.isMetaRegion()) {
+      updateMetaLocation(regionInfo, regionLocation);
+    } else {
+      updateUserRegionLocation(regionInfo, state, regionLocation, lastHost, openSeqNum);
+    }
+  }
+
+  public void updateRegionState(final long openSeqNum, final RegionState newState,
+      final RegionState oldState) throws IOException {
+    updateRegionLocation(newState.getRegion(), newState.getState(), newState.getServerName(),
+        oldState != null ? oldState.getServerName() : null, openSeqNum);
+  }
+
+  protected void updateMetaLocation(final HRegionInfo regionInfo, final ServerName serverName)
+      throws IOException {
+    try {
+      MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName,
+        regionInfo.getReplicaId(), State.OPEN);
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    }
+  }
+
+  protected void updateUserRegionLocation(final HRegionInfo regionInfo, final State state,
+      final ServerName regionLocation, final ServerName lastHost, final long openSeqNum)
+      throws IOException {
+    final int replicaId = regionInfo.getReplicaId();
+    final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo));
+    MetaTableAccessor.addRegionInfo(put, regionInfo);
+    final StringBuilder info = new StringBuilder("Updating hbase:meta row ");
+    info.append(regionInfo.getRegionNameAsString()).append(" with state=").append(state);
+    if (openSeqNum >= 0) {
+      Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
+          "Open region should be on a server");
+      MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, -1, replicaId);
+      info.append(", openSeqNum=").append(openSeqNum);
+      info.append(", server=").append(regionLocation);
+    } else if (regionLocation != null && !regionLocation.equals(lastHost)) {
+      put.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
+          Bytes.toBytes(regionLocation.getServerName()));
+      info.append(", sn=").append(regionLocation);
+    }
+    put.addImmutable(HConstants.CATALOG_FAMILY, getStateColumn(replicaId),
+      Bytes.toBytes(state.name()));
+    LOG.info(info);
+
+    final boolean serialReplication = hasSerialReplicationScope(regionInfo.getTable());
+    if (serialReplication && state == State.OPEN) {
+      Put barrierPut = MetaTableAccessor.makeBarrierPut(regionInfo.getEncodedNameAsBytes(),
+          openSeqNum, regionInfo.getTable().getName());
+      updateRegionLocation(regionInfo, state, put, barrierPut);
+    } else {
+      updateRegionLocation(regionInfo, state, put);
+    }
+  }
+
+  protected void updateRegionLocation(final HRegionInfo regionInfo, final State state,
+      final Put... put) throws IOException {
+    synchronized (this) {
+      if (multiHConnection == null) {
+        multiHConnection = new MultiHConnection(master.getConfiguration(), 1);
+      }
+    }
+
+    try {
+      multiHConnection.processBatchCallback(Arrays.asList(put), TableName.META_TABLE_NAME, null, null);
+    } catch (IOException e) {
+      // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host!
+      // In tests we abort the Master!
+      String msg = String.format("FAILED persisting region=%s state=%s",
+          regionInfo.getShortNameToLog(), state);
+      LOG.error(msg, e);
+      master.abort(msg, e);
+      throw e;
+    }
+  }
+
+  // ============================================================================================
+  //  Update Region Splitting State helpers
+  // ============================================================================================
+  public void splitRegion(final HRegionInfo parent, final HRegionInfo hriA,
+      final HRegionInfo hriB, final ServerName serverName)  throws IOException {
+    final HTableDescriptor htd = getTableDescriptor(parent.getTable());
+    MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName,
+        getRegionReplication(htd), hasSerialReplicationScope(htd));
+  }
+
+  // ============================================================================================
+  //  Update Region Merging State helpers
+  // ============================================================================================
+  public void mergeRegions(final HRegionInfo parent, final HRegionInfo hriA,
+      final HRegionInfo hriB, final ServerName serverName)  throws IOException {
+    final HTableDescriptor htd = getTableDescriptor(parent.getTable());
+    MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
+        getRegionReplication(htd), EnvironmentEdgeManager.currentTime(),
+        hasSerialReplicationScope(htd));
+  }
+
+  // ============================================================================================
+  //  Delete Region State helpers
+  // ============================================================================================
+  public void deleteRegion(final HRegionInfo regionInfo) throws IOException {
+    deleteRegions(Collections.singletonList(regionInfo));
+  }
+
+  public void deleteRegions(final List<HRegionInfo> regions) throws IOException {
+    MetaTableAccessor.deleteRegions(master.getConnection(), regions);
+  }
+
+  // ==========================================================================
+  //  Table Descriptors helpers
+  // ==========================================================================
+  private boolean hasSerialReplicationScope(final TableName tableName) throws IOException {
+    return hasSerialReplicationScope(getTableDescriptor(tableName));
+  }
+
+  private boolean hasSerialReplicationScope(final HTableDescriptor htd) {
+    return (htd != null)? htd.hasSerialReplicationScope(): false;
+  }
+
+  private int getRegionReplication(final TableName tableName) throws IOException {
+    return getRegionReplication(getTableDescriptor(tableName));
+  }
+
+  private int getRegionReplication(final HTableDescriptor htd) {
+    return (htd != null) ? htd.getRegionReplication() : 1;
+  }
+
+  private HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
+    return master.getTableDescriptors().get(tableName);
+  }
+
+  // ==========================================================================
+  //  Server Name
+  // ==========================================================================
+
+  /**
+   * Returns the {@link ServerName} from catalog table {@link Result}
+   * where the region is transitioning. It should be the same as
+   * {@link MetaTableAccessor#getServerName(Result,int)} if the server is at OPEN state.
+   * @param r Result to pull the transitioning server name from
+   * @return A ServerName instance or {@link MetaTableAccessor#getServerName(Result,int)}
+   * if necessary fields not found or empty.
+   */
+  static ServerName getRegionServer(final Result r, int replicaId) {
+    final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        getServerNameColumn(replicaId));
+    if (cell == null || cell.getValueLength() == 0) {
+      RegionLocations locations = MetaTableAccessor.getRegionLocations(r);
+      if (locations != null) {
+        HRegionLocation location = locations.getRegionLocation(replicaId);
+        if (location != null) {
+          return location.getServerName();
+        }
+      }
+      return null;
+    }
+    return ServerName.parseServerName(Bytes.toString(cell.getValueArray(),
+      cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  private static byte[] getServerNameColumn(int replicaId) {
+    return replicaId == 0
+        ? HConstants.SERVERNAME_QUALIFIER
+        : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+          + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+  }
+
+  // ==========================================================================
+  //  Region State
+  // ==========================================================================
+
+  /**
+   * Pull the region state from a catalog table {@link Result}.
+   * @param r Result to pull the region state from
+   * @return the region state, or OPEN if there's no value written.
+   */
+  protected State getRegionState(final Result r, int replicaId) {
+    Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(replicaId));
+    if (cell == null || cell.getValueLength() == 0) return State.OPENING;
+    return State.valueOf(Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  private static byte[] getStateColumn(int replicaId) {
+    return replicaId == 0
+        ? HConstants.STATE_QUALIFIER
+        : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+          + String.format(HRegionInfo.REPLICA_ID_FORMAT, replicaId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
new file mode 100644
index 0000000..24c122d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -0,0 +1,864 @@
+/**
+ *
+ * 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.assignment;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * RegionStates contains a set of maps that describes the in-memory state of the AM, with
+ * the regions available in the system, the region in transition, the offline regions and
+ * the servers holding regions.
+ */
+@InterfaceAudience.Private
+public class RegionStates {
+  private static final Log LOG = LogFactory.getLog(RegionStates.class);
+
+  protected static final State[] STATES_EXPECTEX_IN_OPEN = new State[] {
+    State.OFFLINE, State.CLOSED,      // disable/offline
+    State.SPLITTING, State.SPLIT,     // ServerCrashProcedure
+    State.OPENING, State.FAILED_OPEN, // already in-progress (retrying)
+  };
+
+  protected static final State[] STATES_EXPECTEX_IN_CLOSE = new State[] {
+    State.SPLITTING, State.SPLIT, // ServerCrashProcedure
+    State.OPEN,                   // enabled/open
+    State.CLOSING                 // already in-progress (retrying)
+  };
+
+  private static class AssignmentProcedureEvent extends ProcedureEvent<HRegionInfo> {
+    public AssignmentProcedureEvent(final HRegionInfo regionInfo) {
+      super(regionInfo);
+    }
+  }
+
+  private static class ServerReportEvent extends ProcedureEvent<ServerName> {
+    public ServerReportEvent(final ServerName serverName) {
+      super(serverName);
+    }
+  }
+
+  public static class RegionStateNode implements Comparable<RegionStateNode> {
+    private final HRegionInfo regionInfo;
+    private final ProcedureEvent event;
+
+    private volatile RegionTransitionProcedure procedure = null;
+    private volatile ServerName regionLocation = null;
+    private volatile ServerName lastHost = null;
+    private volatile State state = State.OFFLINE;
+    private volatile long lastUpdate = 0;
+    private volatile long openSeqNum = HConstants.NO_SEQNUM;
+
+    public RegionStateNode(final HRegionInfo regionInfo) {
+      this.regionInfo = regionInfo;
+      this.event = new AssignmentProcedureEvent(regionInfo);
+    }
+
+    public boolean setState(final State update, final State... expected) {
+      final boolean expectedState = isInState(expected);
+      if (expectedState) {
+        this.state = update;
+      }
+      return expectedState;
+    }
+
+    public boolean isInState(final State... expected) {
+      if (expected != null && expected.length > 0) {
+        boolean expectedState = false;
+        for (int i = 0; i < expected.length; ++i) {
+          expectedState |= (state == expected[i]);
+        }
+        return expectedState;
+      }
+      return true;
+    }
+
+    public boolean isStuck() {
+      return isInState(State.FAILED_OPEN) && getProcedure() != null;
+    }
+
+    public boolean isInTransition() {
+      return getProcedure() != null;
+    }
+
+    public long getLastUpdate() {
+      return procedure != null ? procedure.getLastUpdate() : lastUpdate;
+    }
+
+    public void setLastHost(final ServerName serverName) {
+      this.lastHost = serverName;
+    }
+
+    public void setOpenSeqNum(final long seqId) {
+      this.openSeqNum = seqId;
+    }
+
+    public void setRegionLocation(final ServerName serverName) {
+      this.regionLocation = serverName;
+      this.lastUpdate = EnvironmentEdgeManager.currentTime();
+    }
+
+    public boolean setProcedure(final RegionTransitionProcedure proc) {
+      if (this.procedure != null && this.procedure != proc) {
+        return false;
+      }
+      this.procedure = proc;
+      return true;
+    }
+
+    public boolean unsetProcedure(final RegionTransitionProcedure proc) {
+      if (this.procedure != null && this.procedure != proc) {
+        return false;
+      }
+      this.procedure = null;
+      return true;
+    }
+
+    public RegionTransitionProcedure getProcedure() {
+      return procedure;
+    }
+
+    public ProcedureEvent getProcedureEvent() {
+      return event;
+    }
+
+    public HRegionInfo getRegionInfo() {
+      return regionInfo;
+    }
+
+    public TableName getTable() {
+      return getRegionInfo().getTable();
+    }
+
+    public boolean isSystemTable() {
+      return getTable().isSystemTable();
+    }
+
+    public ServerName getLastHost() {
+      return lastHost;
+    }
+
+    public ServerName getRegionLocation() {
+      return regionLocation;
+    }
+
+    public State getState() {
+      return state;
+    }
+
+    public long getOpenSeqNum() {
+      return openSeqNum;
+    }
+
+    public int getFormatVersion() {
+      // we don't have any format for now
+      // it should probably be in regionInfo.getFormatVersion()
+      return 0;
+    }
+
+    @Override
+    public int compareTo(final RegionStateNode other) {
+      // NOTE: HRegionInfo sort by table first, so we are relying on that.
+      // we have a TestRegionState#testOrderedByTable() that check for that.
+      return getRegionInfo().compareTo(other.getRegionInfo());
+    }
+
+    @Override
+    public int hashCode() {
+      return getRegionInfo().hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+      if (this == other) return true;
+      if (!(other instanceof RegionStateNode)) return false;
+      return compareTo((RegionStateNode)other) == 0;
+    }
+
+    @Override
+    public String toString() {
+      return toDescriptiveString();
+    }
+
+    public String toDescriptiveString() {
+      return String.format("state=%s, table=%s, region=%s, server=%s",
+        getState(), getTable(), getRegionInfo().getEncodedName(), getRegionLocation());
+    }
+  }
+
+  // This comparator sorts the RegionStates by time stamp then Region name.
+  // Comparing by timestamp alone can lead us to discard different RegionStates that happen
+  // to share a timestamp.
+  private static class RegionStateStampComparator implements Comparator<RegionState> {
+    @Override
+    public int compare(final RegionState l, final RegionState r) {
+      int stampCmp = Long.compare(l.getStamp(), r.getStamp());
+      return stampCmp != 0 ? stampCmp : l.getRegion().compareTo(r.getRegion());
+    }
+  }
+
+  public enum ServerState { ONLINE, SPLITTING, OFFLINE }
+  public static class ServerStateNode implements Comparable<ServerStateNode> {
+    private final ServerReportEvent reportEvent;
+
+    private final Set<RegionStateNode> regions;
+    private final ServerName serverName;
+
+    private volatile ServerState state = ServerState.ONLINE;
+    private volatile int versionNumber = 0;
+
+    public ServerStateNode(final ServerName serverName) {
+      this.serverName = serverName;
+      this.regions = new HashSet<RegionStateNode>();
+      this.reportEvent = new ServerReportEvent(serverName);
+    }
+
+    public ServerName getServerName() {
+      return serverName;
+    }
+
+    public ServerState getState() {
+      return state;
+    }
+
+    public int getVersionNumber() {
+      return versionNumber;
+    }
+
+    public ProcedureEvent getReportEvent() {
+      return reportEvent;
+    }
+
+    public boolean isInState(final ServerState... expected) {
+      boolean expectedState = false;
+      if (expected != null) {
+        for (int i = 0; i < expected.length; ++i) {
+          expectedState |= (state == expected[i]);
+        }
+      }
+      return expectedState;
+    }
+
+    public void setState(final ServerState state) {
+      this.state = state;
+    }
+
+    public void setVersionNumber(final int versionNumber) {
+      this.versionNumber = versionNumber;
+    }
+
+    public Set<RegionStateNode> getRegions() {
+      return regions;
+    }
+
+    public int getRegionCount() {
+      return regions.size();
+    }
+
+    public ArrayList<HRegionInfo> getRegionInfoList() {
+      ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(regions.size());
+      for (RegionStateNode region: regions) {
+        hris.add(region.getRegionInfo());
+      }
+      return hris;
+    }
+
+    public void addRegion(final RegionStateNode regionNode) {
+      this.regions.add(regionNode);
+    }
+
+    public void removeRegion(final RegionStateNode regionNode) {
+      this.regions.remove(regionNode);
+    }
+
+    @Override
+    public int compareTo(final ServerStateNode other) {
+      return getServerName().compareTo(other.getServerName());
+    }
+
+    @Override
+    public int hashCode() {
+      return getServerName().hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+      if (this == other) return true;
+      if (!(other instanceof ServerStateNode)) return false;
+      return compareTo((ServerStateNode)other) == 0;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("ServerStateNode(%s)", getServerName());
+    }
+  }
+
+  public final static RegionStateStampComparator REGION_STATE_STAMP_COMPARATOR =
+      new RegionStateStampComparator();
+
+  // TODO: Replace the ConcurrentSkipListMaps
+  private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
+      new ConcurrentSkipListMap<byte[], RegionStateNode>(Bytes.BYTES_COMPARATOR);
+
+  private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionInTransition =
+    new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
+
+  private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionOffline =
+    new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
+
+  private final ConcurrentSkipListMap<byte[], RegionFailedOpen> regionFailedOpen =
+    new ConcurrentSkipListMap<byte[], RegionFailedOpen>(Bytes.BYTES_COMPARATOR);
+
+  private final ConcurrentHashMap<ServerName, ServerStateNode> serverMap =
+      new ConcurrentHashMap<ServerName, ServerStateNode>();
+
+  public RegionStates() { }
+
+  public void clear() {
+    regionsMap.clear();
+    regionInTransition.clear();
+    regionOffline.clear();
+    serverMap.clear();
+  }
+
+  // ==========================================================================
+  //  RegionStateNode helpers
+  // ==========================================================================
+  protected RegionStateNode createRegionNode(final HRegionInfo regionInfo) {
+    RegionStateNode newNode = new RegionStateNode(regionInfo);
+    RegionStateNode oldNode = regionsMap.putIfAbsent(regionInfo.getRegionName(), newNode);
+    return oldNode != null ? oldNode : newNode;
+  }
+
+  protected RegionStateNode getOrCreateRegionNode(final HRegionInfo regionInfo) {
+    RegionStateNode node = regionsMap.get(regionInfo.getRegionName());
+    return node != null ? node : createRegionNode(regionInfo);
+  }
+
+  public RegionStateNode getRegionNodeFromName(final byte[] regionName) {
+    return regionsMap.get(regionName);
+  }
+
+  protected RegionStateNode getRegionNode(final HRegionInfo regionInfo) {
+    return getRegionNodeFromName(regionInfo.getRegionName());
+  }
+
+  public RegionStateNode getRegionNodeFromEncodedName(final String encodedRegionName) {
+    // TODO: Need a map <encodedName, ...> but it is just dispatch merge...
+    for (RegionStateNode node: regionsMap.values()) {
+      if (node.getRegionInfo().getEncodedName().equals(encodedRegionName)) {
+        return node;
+      }
+    }
+    return null;
+  }
+
+  public void deleteRegion(final HRegionInfo regionInfo) {
+    regionsMap.remove(regionInfo.getRegionName());
+  }
+
+  public ArrayList<RegionStateNode> getTableRegionStateNodes(final TableName tableName) {
+    final ArrayList<RegionStateNode> regions = new ArrayList<RegionStateNode>();
+    for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
+      if (!node.getTable().equals(tableName)) break;
+      regions.add(node);
+    }
+    return regions;
+  }
+
+  public ArrayList<RegionState> getTableRegionStates(final TableName tableName) {
+    final ArrayList<RegionState> regions = new ArrayList<RegionState>();
+    for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
+      if (!node.getTable().equals(tableName)) break;
+      regions.add(createRegionState(node));
+    }
+    return regions;
+  }
+
+  public ArrayList<HRegionInfo> getTableRegionsInfo(final TableName tableName) {
+    final ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
+    for (RegionStateNode node: regionsMap.tailMap(tableName.getName()).values()) {
+      if (!node.getTable().equals(tableName)) break;
+      regions.add(node.getRegionInfo());
+    }
+    return regions;
+  }
+
+  public Collection<RegionStateNode> getRegionNodes() {
+    return regionsMap.values();
+  }
+
+  public ArrayList<RegionState> getRegionStates() {
+    final ArrayList<RegionState> regions = new ArrayList<RegionState>(regionsMap.size());
+    for (RegionStateNode node: regionsMap.values()) {
+      regions.add(createRegionState(node));
+    }
+    return regions;
+  }
+
+  // ==========================================================================
+  //  RegionState helpers
+  // ==========================================================================
+  public RegionState getRegionState(final HRegionInfo regionInfo) {
+    return createRegionState(getRegionNode(regionInfo));
+  }
+
+  public RegionState getRegionState(final String encodedRegionName) {
+    return createRegionState(getRegionNodeFromEncodedName(encodedRegionName));
+  }
+
+  private RegionState createRegionState(final RegionStateNode node) {
+    return node == null ? null :
+      new RegionState(node.getRegionInfo(), node.getState(),
+        node.getLastUpdate(), node.getRegionLocation());
+  }
+
+  // ============================================================================================
+  //  TODO: helpers
+  // ============================================================================================
+  public boolean hasTableRegionStates(final TableName tableName) {
+    // TODO
+    return getTableRegionStates(tableName).size() > 0;
+  }
+
+  public List<HRegionInfo> getRegionsOfTable(final TableName table) {
+    return getRegionsOfTable(table, false);
+  }
+
+  public List<HRegionInfo> getRegionsOfTable(final TableName table, final boolean includeOffline) {
+    final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table);
+    final ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(nodes.size());
+    for (RegionStateNode node: nodes) {
+      final HRegionInfo hri = node.getRegionInfo();
+      if ((!hri.isOffline() && !hri.isSplit()) ||
+          ((hri.isOffline() || hri.isSplit()) && includeOffline)) {
+        hris.add(hri);
+      }
+    }
+    return hris;
+  }
+
+  /**
+   * Returns the set of regions hosted by the specified server
+   * @param serverName the server we are interested in
+   * @return set of HRegionInfo hosted by the specified server
+   */
+  public List<HRegionInfo> getServerRegionInfoSet(final ServerName serverName) {
+    final ServerStateNode serverInfo = getServerNode(serverName);
+    if (serverInfo == null) return Collections.emptyList();
+
+    synchronized (serverInfo) {
+      return serverInfo.getRegionInfoList();
+    }
+  }
+
+  // ============================================================================================
+  //  TODO: split helpers
+  // ============================================================================================
+  public void logSplit(final ServerName serverName) {
+    final ServerStateNode serverNode = getOrCreateServer(serverName);
+    synchronized (serverNode) {
+      serverNode.setState(ServerState.SPLITTING);
+      for (RegionStateNode regionNode: serverNode.getRegions()) {
+        synchronized (regionNode) {
+          // TODO: Abort procedure if present
+          regionNode.setState(State.SPLITTING);
+        }
+      }
+    }
+  }
+
+  public void logSplit(final HRegionInfo regionInfo) {
+    final RegionStateNode regionNode = getRegionNode(regionInfo);
+    synchronized (regionNode) {
+      regionNode.setState(State.SPLIT);
+    }
+  }
+
+  @VisibleForTesting
+  public void updateRegionState(final HRegionInfo regionInfo, final State state) {
+    final RegionStateNode regionNode = getOrCreateRegionNode(regionInfo);
+    synchronized (regionNode) {
+      regionNode.setState(state);
+    }
+  }
+
+  // ============================================================================================
+  //  TODO:
+  // ============================================================================================
+  public List<HRegionInfo> getAssignedRegions() {
+    final List<HRegionInfo> result = new ArrayList<HRegionInfo>();
+    for (RegionStateNode node: regionsMap.values()) {
+      if (!node.isInTransition()) {
+        result.add(node.getRegionInfo());
+      }
+    }
+    return result;
+  }
+
+  public boolean isRegionInState(final HRegionInfo regionInfo, final State... state) {
+    final RegionStateNode region = getRegionNode(regionInfo);
+    if (region != null) {
+      synchronized (region) {
+        return region.isInState(state);
+      }
+    }
+    return false;
+  }
+
+  public boolean isRegionOnline(final HRegionInfo regionInfo) {
+    return isRegionInState(regionInfo, State.OPEN);
+  }
+
+  public Map<ServerName, List<HRegionInfo>> getSnapShotOfAssignment(
+      final Collection<HRegionInfo> regions) {
+    final Map<ServerName, List<HRegionInfo>> result = new HashMap<ServerName, List<HRegionInfo>>();
+    for (HRegionInfo hri: regions) {
+      final RegionStateNode node = getRegionNode(hri);
+      if (node == null) continue;
+
+      // TODO: State.OPEN
+      final ServerName serverName = node.getRegionLocation();
+      if (serverName == null) continue;
+
+      List<HRegionInfo> serverRegions = result.get(serverName);
+      if (serverRegions == null) {
+        serverRegions = new ArrayList<HRegionInfo>();
+        result.put(serverName, serverRegions);
+      }
+
+      serverRegions.add(node.getRegionInfo());
+    }
+    return result;
+  }
+
+  public Map<HRegionInfo, ServerName> getRegionAssignments() {
+    final HashMap<HRegionInfo, ServerName> assignments = new HashMap<HRegionInfo, ServerName>();
+    for (RegionStateNode node: regionsMap.values()) {
+      assignments.put(node.getRegionInfo(), node.getRegionLocation());
+    }
+    return assignments;
+  }
+
+  public Map<RegionState.State, List<HRegionInfo>> getRegionByStateOfTable(TableName tableName) {
+    final State[] states = State.values();
+    final Map<RegionState.State, List<HRegionInfo>> tableRegions =
+        new HashMap<State, List<HRegionInfo>>(states.length);
+    for (int i = 0; i < states.length; ++i) {
+      tableRegions.put(states[i], new ArrayList<HRegionInfo>());
+    }
+
+    for (RegionStateNode node: regionsMap.values()) {
+      tableRegions.get(node.getState()).add(node.getRegionInfo());
+    }
+    return tableRegions;
+  }
+
+  public ServerName getRegionServerOfRegion(final HRegionInfo regionInfo) {
+    final RegionStateNode region = getRegionNode(regionInfo);
+    if (region != null) {
+      synchronized (region) {
+        ServerName server = region.getRegionLocation();
+        return server != null ? server : region.getLastHost();
+      }
+    }
+    return null;
+  }
+
+  /**
+   * This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
+   * Can't let out original since it can change and at least the load balancer
+   * wants to iterate this exported list.  We need to synchronize on regions
+   * since all access to this.servers is under a lock on this.regions.
+   * @param forceByCluster a flag to force to aggregate the server-load to the cluster level
+   * @return A clone of current assignments by table.
+   */
+  public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable(
+      final boolean forceByCluster) {
+    if (!forceByCluster) return getAssignmentsByTable();
+
+    final HashMap<ServerName, List<HRegionInfo>> ensemble =
+      new HashMap<ServerName, List<HRegionInfo>>(serverMap.size());
+    for (ServerStateNode serverNode: serverMap.values()) {
+      ensemble.put(serverNode.getServerName(), serverNode.getRegionInfoList());
+    }
+
+    // TODO: can we use Collections.singletonMap(HConstants.ENSEMBLE_TABLE_NAME, ensemble)?
+    final Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
+      new HashMap<TableName, Map<ServerName, List<HRegionInfo>>>(1);
+    result.put(HConstants.ENSEMBLE_TABLE_NAME, ensemble);
+    return result;
+  }
+
+  public Map<TableName, Map<ServerName, List<HRegionInfo>>> getAssignmentsByTable() {
+    final Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
+      new HashMap<TableName, Map<ServerName, List<HRegionInfo>>>();
+    for (RegionStateNode node: regionsMap.values()) {
+      Map<ServerName, List<HRegionInfo>> tableResult = result.get(node.getTable());
+      if (tableResult == null) {
+        tableResult = new HashMap<ServerName, List<HRegionInfo>>();
+        result.put(node.getTable(), tableResult);
+      }
+
+      final ServerName serverName = node.getRegionLocation();
+      List<HRegionInfo> serverResult = tableResult.get(serverName);
+      if (serverResult == null) {
+        serverResult = new ArrayList<HRegionInfo>();
+        tableResult.put(serverName, serverResult);
+      }
+
+      serverResult.add(node.getRegionInfo());
+    }
+    return result;
+  }
+
+  // ==========================================================================
+  //  Region in transition helpers
+  // ==========================================================================
+  protected boolean addRegionInTransition(final RegionStateNode regionNode,
+      final RegionTransitionProcedure procedure) {
+    if (procedure != null && !regionNode.setProcedure(procedure)) return false;
+
+    regionInTransition.put(regionNode.getRegionInfo(), regionNode);
+    return true;
+  }
+
+  protected void removeRegionInTransition(final RegionStateNode regionNode,
+      final RegionTransitionProcedure procedure) {
+    regionInTransition.remove(regionNode.getRegionInfo());
+    regionNode.unsetProcedure(procedure);
+  }
+
+  public boolean hasRegionsInTransition() {
+    return !regionInTransition.isEmpty();
+  }
+
+  public boolean isRegionInTransition(final HRegionInfo regionInfo) {
+    final RegionStateNode node = regionInTransition.get(regionInfo);
+    return node != null ? node.isInTransition() : false;
+  }
+
+  public RegionState getRegionTransitionState(final HRegionInfo hri) {
+    RegionStateNode node = regionInTransition.get(hri);
+    if (node == null) return null;
+
+    synchronized (node) {
+      return node.isInTransition() ? createRegionState(node) : null;
+    }
+  }
+
+  public List<RegionStateNode> getRegionsInTransition() {
+    return new ArrayList<RegionStateNode>(regionInTransition.values());
+  }
+
+  /**
+   * Get the number of regions in transition.
+   */
+  public int getRegionsInTransitionCount() {
+    return regionInTransition.size();
+  }
+
+  public List<RegionState> getRegionsStateInTransition() {
+    final List<RegionState> rit = new ArrayList<RegionState>(regionInTransition.size());
+    for (RegionStateNode node: regionInTransition.values()) {
+      rit.add(createRegionState(node));
+    }
+    return rit;
+  }
+
+  public SortedSet<RegionState> getRegionsInTransitionOrderedByTimestamp() {
+    final SortedSet<RegionState> rit = new TreeSet<RegionState>(REGION_STATE_STAMP_COMPARATOR);
+    for (RegionStateNode node: regionInTransition.values()) {
+      rit.add(createRegionState(node));
+    }
+    return rit;
+  }
+
+  // ==========================================================================
+  //  Region offline helpers
+  // ==========================================================================
+  public void addToOfflineRegions(final RegionStateNode regionNode) {
+    regionOffline.put(regionNode.getRegionInfo(), regionNode);
+  }
+
+  public void removeFromOfflineRegions(final HRegionInfo regionInfo) {
+    regionOffline.remove(regionInfo);
+  }
+
+  // ==========================================================================
+  //  Region FAIL_OPEN helpers
+  // ==========================================================================
+  public static final class RegionFailedOpen {
+    private final RegionStateNode regionNode;
+
+    private volatile Exception exception = null;
+    private volatile int retries = 0;
+
+    public RegionFailedOpen(final RegionStateNode regionNode) {
+      this.regionNode = regionNode;
+    }
+
+    public RegionStateNode getRegionNode() {
+      return regionNode;
+    }
+
+    public HRegionInfo getRegionInfo() {
+      return regionNode.getRegionInfo();
+    }
+
+    public int incrementAndGetRetries() {
+      return ++this.retries;
+    }
+
+    public int getRetries() {
+      return retries;
+    }
+
+    public void setException(final Exception exception) {
+      this.exception = exception;
+    }
+
+    public Exception getException() {
+      return this.exception;
+    }
+  }
+
+  public RegionFailedOpen addToFailedOpen(final RegionStateNode regionNode) {
+    final byte[] key = regionNode.getRegionInfo().getRegionName();
+    RegionFailedOpen node = regionFailedOpen.get(key);
+    if (node == null) {
+      RegionFailedOpen newNode = new RegionFailedOpen(regionNode);
+      RegionFailedOpen oldNode = regionFailedOpen.putIfAbsent(key, newNode);
+      node = oldNode != null ? oldNode : newNode;
+    }
+    return node;
+  }
+
+  public RegionFailedOpen getFailedOpen(final HRegionInfo regionInfo) {
+    return regionFailedOpen.get(regionInfo.getRegionName());
+  }
+
+  public void removeFromFailedOpen(final HRegionInfo regionInfo) {
+    regionFailedOpen.remove(regionInfo.getRegionName());
+  }
+
+  public List<RegionState> getRegionFailedOpen() {
+    if (regionFailedOpen.isEmpty()) return Collections.emptyList();
+
+    ArrayList<RegionState> regions = new ArrayList<RegionState>(regionFailedOpen.size());
+    for (RegionFailedOpen r: regionFailedOpen.values()) {
+      regions.add(createRegionState(r.getRegionNode()));
+    }
+    return regions;
+  }
+
+  // ==========================================================================
+  //  Servers
+  // ==========================================================================
+  public ServerStateNode getOrCreateServer(final ServerName serverName) {
+    ServerStateNode node = serverMap.get(serverName);
+    if (node == null) {
+      node = new ServerStateNode(serverName);
+      ServerStateNode oldNode = serverMap.putIfAbsent(serverName, node);
+      node = oldNode != null ? oldNode : node;
+    }
+    return node;
+  }
+
+  public void removeServer(final ServerName serverName) {
+    serverMap.remove(serverName);
+  }
+
+  protected ServerStateNode getServerNode(final ServerName serverName) {
+    return serverMap.get(serverName);
+  }
+
+  public double getAverageLoad() {
+    int numServers = 0;
+    int totalLoad = 0;
+    for (ServerStateNode node: serverMap.values()) {
+      // TODO: Not used!
+      int regionCount = node.getRegionCount();
+      totalLoad++;
+      numServers++;
+    }
+    return numServers == 0 ? 0.0 : (double)totalLoad / (double)numServers;
+  }
+
+  public ServerStateNode addRegionToServer(final ServerName serverName,
+      final RegionStateNode regionNode) {
+    ServerStateNode serverNode = getOrCreateServer(serverName);
+    serverNode.addRegion(regionNode);
+    return serverNode;
+  }
+
+  public ServerStateNode removeRegionFromServer(final ServerName serverName,
+      final RegionStateNode regionNode) {
+    ServerStateNode serverNode = getOrCreateServer(serverName);
+    serverNode.removeRegion(regionNode);
+    return serverNode;
+  }
+
+  // ==========================================================================
+  //  ToString helpers
+  // ==========================================================================
+  public static String regionNamesToString(final Collection<byte[]> regions) {
+    final StringBuilder sb = new StringBuilder();
+    final Iterator<byte[]> it = regions.iterator();
+    sb.append("[");
+    if (it.hasNext()) {
+      sb.append(Bytes.toStringBinary(it.next()));
+      while (it.hasNext()) {
+        sb.append(", ");
+        sb.append(Bytes.toStringBinary(it.next()));
+      }
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+}


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index f4f6a8c..88fc8f6 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -1716,100 +1716,201 @@ public final class MasterProcedureProtos {
   }
 
   /**
-   * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
+   * Protobuf enum {@code hbase.pb.DispatchMergingRegionsState}
    */
-  public enum MergeTableRegionsState
+  public enum DispatchMergingRegionsState
       implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
+     * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
      */
-    MERGE_TABLE_REGIONS_PREPARE(1),
+    DISPATCH_MERGING_REGIONS_PREPARE(1),
     /**
-     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
+     * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
      */
-    MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(2),
+    DISPATCH_MERGING_REGIONS_PRE_OPERATION(2),
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
+     * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
      */
-    MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(3),
+    DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS(3),
     /**
-     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
+     * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
      */
-    MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(4),
+    DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS(4),
     /**
-     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
+     * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
      */
-    MERGE_TABLE_REGIONS_CLOSE_REGIONS(5),
+    DISPATCH_MERGING_REGIONS_POST_OPERATION(5),
+    ;
+
     /**
-     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
+     * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
      */
-    MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(6),
+    public static final int DISPATCH_MERGING_REGIONS_PREPARE_VALUE = 1;
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
+     * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
      */
-    MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(7),
+    public static final int DISPATCH_MERGING_REGIONS_PRE_OPERATION_VALUE = 2;
     /**
-     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
+     * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
      */
-    MERGE_TABLE_REGIONS_UPDATE_META(8),
+    public static final int DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
+     * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
      */
-    MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(9),
+    public static final int DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS_VALUE = 4;
     /**
-     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
+     * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
      */
-    MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(10),
+    public static final int DISPATCH_MERGING_REGIONS_POST_OPERATION_VALUE = 5;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
+     * @deprecated Use {@link #forNumber(int)} instead.
      */
-    MERGE_TABLE_REGIONS_POST_OPERATION(11),
-    ;
+    @java.lang.Deprecated
+    public static DispatchMergingRegionsState valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static DispatchMergingRegionsState forNumber(int value) {
+      switch (value) {
+        case 1: return DISPATCH_MERGING_REGIONS_PREPARE;
+        case 2: return DISPATCH_MERGING_REGIONS_PRE_OPERATION;
+        case 3: return DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS;
+        case 4: return DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS;
+        case 5: return DISPATCH_MERGING_REGIONS_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<DispatchMergingRegionsState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        DispatchMergingRegionsState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<DispatchMergingRegionsState>() {
+            public DispatchMergingRegionsState findValueByNumber(int number) {
+              return DispatchMergingRegionsState.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(14);
+    }
+
+    private static final DispatchMergingRegionsState[] VALUES = values();
+
+    public static DispatchMergingRegionsState valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private DispatchMergingRegionsState(int value) {
+      this.value = value;
+    }
 
+    // @@protoc_insertion_point(enum_scope:hbase.pb.DispatchMergingRegionsState)
+  }
+
+  /**
+   * Protobuf enum {@code hbase.pb.SplitTableRegionState}
+   */
+  public enum SplitTableRegionState
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
+     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_PREPARE_VALUE = 1;
+    SPLIT_TABLE_REGION_PREPARE(1),
+    /**
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
+     */
+    SPLIT_TABLE_REGION_PRE_OPERATION(2),
+    /**
+     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 3;</code>
+     */
+    SPLIT_TABLE_REGION_CLOSE_PARENT_REGION(3),
+    /**
+     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 4;</code>
+     */
+    SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS(4),
+    /**
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 5;</code>
+     */
+    SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR(5),
+    /**
+     * <code>SPLIT_TABLE_REGION_UPDATE_META = 6;</code>
+     */
+    SPLIT_TABLE_REGION_UPDATE_META(6),
+    /**
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 7;</code>
+     */
+    SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR(7),
+    /**
+     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 8;</code>
+     */
+    SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS(8),
     /**
-     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
+     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 9;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 2;
+    SPLIT_TABLE_REGION_POST_OPERATION(9),
+    ;
+
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
+     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION_VALUE = 3;
+    public static final int SPLIT_TABLE_REGION_PREPARE_VALUE = 1;
     /**
-     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE_VALUE = 4;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_VALUE = 2;
     /**
-     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
+     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 3;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_CLOSE_REGIONS_VALUE = 5;
+    public static final int SPLIT_TABLE_REGION_CLOSE_PARENT_REGION_VALUE = 3;
     /**
-     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
+     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 4;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_CREATE_MERGED_REGION_VALUE = 6;
+    public static final int SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS_VALUE = 4;
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 5;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION_VALUE = 7;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR_VALUE = 5;
     /**
-     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
+     * <code>SPLIT_TABLE_REGION_UPDATE_META = 6;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_UPDATE_META_VALUE = 8;
+    public static final int SPLIT_TABLE_REGION_UPDATE_META_VALUE = 6;
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 7;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION_VALUE = 9;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR_VALUE = 7;
     /**
-     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
+     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 8;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_OPEN_MERGED_REGION_VALUE = 10;
+    public static final int SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS_VALUE = 8;
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
+     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 9;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_POST_OPERATION_VALUE = 11;
+    public static final int SPLIT_TABLE_REGION_POST_OPERATION_VALUE = 9;
 
 
     public final int getNumber() {
@@ -1820,36 +1921,34 @@ public final class MasterProcedureProtos {
      * @deprecated Use {@link #forNumber(int)} instead.
      */
     @java.lang.Deprecated
-    public static MergeTableRegionsState valueOf(int value) {
+    public static SplitTableRegionState valueOf(int value) {
       return forNumber(value);
     }
 
-    public static MergeTableRegionsState forNumber(int value) {
+    public static SplitTableRegionState forNumber(int value) {
       switch (value) {
-        case 1: return MERGE_TABLE_REGIONS_PREPARE;
-        case 2: return MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS;
-        case 3: return MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION;
-        case 4: return MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE;
-        case 5: return MERGE_TABLE_REGIONS_CLOSE_REGIONS;
-        case 6: return MERGE_TABLE_REGIONS_CREATE_MERGED_REGION;
-        case 7: return MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION;
-        case 8: return MERGE_TABLE_REGIONS_UPDATE_META;
-        case 9: return MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION;
-        case 10: return MERGE_TABLE_REGIONS_OPEN_MERGED_REGION;
-        case 11: return MERGE_TABLE_REGIONS_POST_OPERATION;
+        case 1: return SPLIT_TABLE_REGION_PREPARE;
+        case 2: return SPLIT_TABLE_REGION_PRE_OPERATION;
+        case 3: return SPLIT_TABLE_REGION_CLOSE_PARENT_REGION;
+        case 4: return SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS;
+        case 5: return SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR;
+        case 6: return SPLIT_TABLE_REGION_UPDATE_META;
+        case 7: return SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR;
+        case 8: return SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS;
+        case 9: return SPLIT_TABLE_REGION_POST_OPERATION;
         default: return null;
       }
     }
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SplitTableRegionState>
         internalGetValueMap() {
       return internalValueMap;
     }
     private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
-        MergeTableRegionsState> internalValueMap =
-          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>() {
-            public MergeTableRegionsState findValueByNumber(int number) {
-              return MergeTableRegionsState.forNumber(number);
+        SplitTableRegionState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SplitTableRegionState>() {
+            public SplitTableRegionState findValueByNumber(int number) {
+              return SplitTableRegionState.forNumber(number);
             }
           };
 
@@ -1863,12 +1962,12 @@ public final class MasterProcedureProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(14);
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
     }
 
-    private static final MergeTableRegionsState[] VALUES = values();
+    private static final SplitTableRegionState[] VALUES = values();
 
-    public static MergeTableRegionsState valueOf(
+    public static SplitTableRegionState valueOf(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
@@ -1879,100 +1978,116 @@ public final class MasterProcedureProtos {
 
     private final int value;
 
-    private MergeTableRegionsState(int value) {
+    private SplitTableRegionState(int value) {
       this.value = value;
     }
 
-    // @@protoc_insertion_point(enum_scope:hbase.pb.MergeTableRegionsState)
+    // @@protoc_insertion_point(enum_scope:hbase.pb.SplitTableRegionState)
   }
 
   /**
-   * Protobuf enum {@code hbase.pb.SplitTableRegionState}
+   * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
    */
-  public enum SplitTableRegionState
+  public enum MergeTableRegionsState
       implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
+     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
      */
-    SPLIT_TABLE_REGION_PREPARE(1),
+    MERGE_TABLE_REGIONS_PREPARE(1),
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_OPERATION = 2;</code>
      */
-    SPLIT_TABLE_REGION_PRE_OPERATION(2),
+    MERGE_TABLE_REGIONS_PRE_OPERATION(2),
+    /**
+     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+     */
+    MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(3),
     /**
-     * <code>SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 4;</code>
      */
-    SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE(3),
+    MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(4),
     /**
-     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
+     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 5;</code>
      */
-    SPLIT_TABLE_REGION_CLOSE_PARENT_REGION(4),
+    MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(5),
     /**
-     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
+     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 6;</code>
      */
-    SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS(5),
+    MERGE_TABLE_REGIONS_CLOSE_REGIONS(6),
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;</code>
+     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 7;</code>
      */
-    SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR(6),
+    MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(7),
     /**
-     * <code>SPLIT_TABLE_REGION_UPDATE_META = 7;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 8;</code>
      */
-    SPLIT_TABLE_REGION_UPDATE_META(7),
+    MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(8),
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;</code>
+     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 9;</code>
      */
-    SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR(8),
+    MERGE_TABLE_REGIONS_UPDATE_META(9),
     /**
-     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;</code>
+     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 10;</code>
      */
-    SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS(9),
+    MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(10),
     /**
-     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 10;</code>
+     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 11;</code>
      */
-    SPLIT_TABLE_REGION_POST_OPERATION(10),
+    MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(11),
+    /**
+     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 12;</code>
+     */
+    MERGE_TABLE_REGIONS_POST_OPERATION(12),
     ;
 
     /**
-     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
+     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PREPARE_VALUE = 1;
+    public static final int MERGE_TABLE_REGIONS_PREPARE_VALUE = 1;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_OPERATION = 2;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_VALUE = 2;
+    public static final int MERGE_TABLE_REGIONS_PRE_OPERATION_VALUE = 2;
     /**
-     * <code>SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;</code>
+     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
      */
-    public static final int SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE_VALUE = 3;
+    public static final int MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
     /**
-     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 4;</code>
      */
-    public static final int SPLIT_TABLE_REGION_CLOSE_PARENT_REGION_VALUE = 4;
+    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION_VALUE = 4;
     /**
-     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
+     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 5;</code>
      */
-    public static final int SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS_VALUE = 5;
+    public static final int MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE_VALUE = 5;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;</code>
+     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 6;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR_VALUE = 6;
+    public static final int MERGE_TABLE_REGIONS_CLOSE_REGIONS_VALUE = 6;
     /**
-     * <code>SPLIT_TABLE_REGION_UPDATE_META = 7;</code>
+     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 7;</code>
      */
-    public static final int SPLIT_TABLE_REGION_UPDATE_META_VALUE = 7;
+    public static final int MERGE_TABLE_REGIONS_CREATE_MERGED_REGION_VALUE = 7;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 8;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR_VALUE = 8;
+    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION_VALUE = 8;
     /**
-     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;</code>
+     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 9;</code>
      */
-    public static final int SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS_VALUE = 9;
+    public static final int MERGE_TABLE_REGIONS_UPDATE_META_VALUE = 9;
     /**
-     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 10;</code>
+     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 10;</code>
      */
-    public static final int SPLIT_TABLE_REGION_POST_OPERATION_VALUE = 10;
+    public static final int MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION_VALUE = 10;
+    /**
+     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 11;</code>
+     */
+    public static final int MERGE_TABLE_REGIONS_OPEN_MERGED_REGION_VALUE = 11;
+    /**
+     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 12;</code>
+     */
+    public static final int MERGE_TABLE_REGIONS_POST_OPERATION_VALUE = 12;
 
 
     public final int getNumber() {
@@ -1983,35 +2098,37 @@ public final class MasterProcedureProtos {
      * @deprecated Use {@link #forNumber(int)} instead.
      */
     @java.lang.Deprecated
-    public static SplitTableRegionState valueOf(int value) {
+    public static MergeTableRegionsState valueOf(int value) {
       return forNumber(value);
     }
 
-    public static SplitTableRegionState forNumber(int value) {
+    public static MergeTableRegionsState forNumber(int value) {
       switch (value) {
-        case 1: return SPLIT_TABLE_REGION_PREPARE;
-        case 2: return SPLIT_TABLE_REGION_PRE_OPERATION;
-        case 3: return SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE;
-        case 4: return SPLIT_TABLE_REGION_CLOSE_PARENT_REGION;
-        case 5: return SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS;
-        case 6: return SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR;
-        case 7: return SPLIT_TABLE_REGION_UPDATE_META;
-        case 8: return SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR;
-        case 9: return SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS;
-        case 10: return SPLIT_TABLE_REGION_POST_OPERATION;
+        case 1: return MERGE_TABLE_REGIONS_PREPARE;
+        case 2: return MERGE_TABLE_REGIONS_PRE_OPERATION;
+        case 3: return MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS;
+        case 4: return MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION;
+        case 5: return MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE;
+        case 6: return MERGE_TABLE_REGIONS_CLOSE_REGIONS;
+        case 7: return MERGE_TABLE_REGIONS_CREATE_MERGED_REGION;
+        case 8: return MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION;
+        case 9: return MERGE_TABLE_REGIONS_UPDATE_META;
+        case 10: return MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION;
+        case 11: return MERGE_TABLE_REGIONS_OPEN_MERGED_REGION;
+        case 12: return MERGE_TABLE_REGIONS_POST_OPERATION;
         default: return null;
       }
     }
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SplitTableRegionState>
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>
         internalGetValueMap() {
       return internalValueMap;
     }
     private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
-        SplitTableRegionState> internalValueMap =
-          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<SplitTableRegionState>() {
-            public SplitTableRegionState findValueByNumber(int number) {
-              return SplitTableRegionState.forNumber(number);
+        MergeTableRegionsState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MergeTableRegionsState>() {
+            public MergeTableRegionsState findValueByNumber(int number) {
+              return MergeTableRegionsState.forNumber(number);
             }
           };
 
@@ -2025,12 +2142,12 @@ public final class MasterProcedureProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(15);
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(16);
     }
 
-    private static final SplitTableRegionState[] VALUES = values();
+    private static final MergeTableRegionsState[] VALUES = values();
 
-    public static SplitTableRegionState valueOf(
+    public static MergeTableRegionsState valueOf(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
       if (desc.getType() != getDescriptor()) {
         throw new java.lang.IllegalArgumentException(
@@ -2041,11 +2158,11 @@ public final class MasterProcedureProtos {
 
     private final int value;
 
-    private SplitTableRegionState(int value) {
+    private MergeTableRegionsState(int value) {
       this.value = value;
     }
 
-    // @@protoc_insertion_point(enum_scope:hbase.pb.SplitTableRegionState)
+    // @@protoc_insertion_point(enum_scope:hbase.pb.MergeTableRegionsState)
   }
 
   /**
@@ -2186,7 +2303,7 @@ public final class MasterProcedureProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(16);
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(17);
     }
 
     private static final ServerCrashState[] VALUES = values();
@@ -2209,59 +2326,248 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.ServerCrashState)
   }
 
-  public interface CreateTableStateDataOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableStateData)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
+  /**
+   * Protobuf enum {@code hbase.pb.RegionTransitionState}
+   */
+  public enum RegionTransitionState
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     * <code>REGION_TRANSITION_QUEUE = 1;</code>
      */
-    boolean hasUserInfo();
+    REGION_TRANSITION_QUEUE(1),
     /**
-     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     * <code>REGION_TRANSITION_DISPATCH = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    REGION_TRANSITION_DISPATCH(2),
     /**
-     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     * <code>REGION_TRANSITION_FINISH = 3;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+    REGION_TRANSITION_FINISH(3),
+    ;
 
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     * <code>REGION_TRANSITION_QUEUE = 1;</code>
      */
-    boolean hasTableSchema();
+    public static final int REGION_TRANSITION_QUEUE_VALUE = 1;
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     * <code>REGION_TRANSITION_DISPATCH = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
+    public static final int REGION_TRANSITION_DISPATCH_VALUE = 2;
     /**
-     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     * <code>REGION_TRANSITION_FINISH = 3;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
+    public static final int REGION_TRANSITION_FINISH_VALUE = 3;
+
+
+    public final int getNumber() {
+      return value;
+    }
 
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * @deprecated Use {@link #forNumber(int)} instead.
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
-        getRegionInfoList();
+    @java.lang.Deprecated
+    public static RegionTransitionState valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static RegionTransitionState forNumber(int value) {
+      switch (value) {
+        case 1: return REGION_TRANSITION_QUEUE;
+        case 2: return REGION_TRANSITION_DISPATCH;
+        case 3: return REGION_TRANSITION_FINISH;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<RegionTransitionState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        RegionTransitionState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<RegionTransitionState>() {
+            public RegionTransitionState findValueByNumber(int number) {
+              return RegionTransitionState.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(18);
+    }
+
+    private static final RegionTransitionState[] VALUES = values();
+
+    public static RegionTransitionState valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private RegionTransitionState(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.RegionTransitionState)
+  }
+
+  /**
+   * Protobuf enum {@code hbase.pb.MoveRegionState}
+   */
+  public enum MoveRegionState
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>MOVE_REGION_UNASSIGN = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    MOVE_REGION_UNASSIGN(1),
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>MOVE_REGION_ASSIGN = 2;</code>
      */
-    int getRegionInfoCount();
+    MOVE_REGION_ASSIGN(2),
+    ;
+
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>MOVE_REGION_UNASSIGN = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
-        getRegionInfoOrBuilderList();
+    public static final int MOVE_REGION_UNASSIGN_VALUE = 1;
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>MOVE_REGION_ASSIGN = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
-        int index);
+    public static final int MOVE_REGION_ASSIGN_VALUE = 2;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static MoveRegionState valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static MoveRegionState forNumber(int value) {
+      switch (value) {
+        case 1: return MOVE_REGION_UNASSIGN;
+        case 2: return MOVE_REGION_ASSIGN;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MoveRegionState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        MoveRegionState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<MoveRegionState>() {
+            public MoveRegionState findValueByNumber(int number) {
+              return MoveRegionState.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(19);
+    }
+
+    private static final MoveRegionState[] VALUES = values();
+
+    public static MoveRegionState valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private MoveRegionState(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.MoveRegionState)
+  }
+
+  public interface CreateTableStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableStateData)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    /**
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     */
+    boolean hasTableSchema();
+    /**
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
+    /**
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
+
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code hbase.pb.CreateTableStateData}
@@ -19892,8 +20198,8 @@ public final class MasterProcedureProtos {
 
   }
 
-  public interface MergeTableRegionsStateDataOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsStateData)
+  public interface DispatchMergingRegionsStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsStateData)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
@@ -19910,63 +20216,63 @@ public final class MasterProcedureProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
 
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    /**
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
         getRegionInfoList();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     int getRegionInfoCount();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
         getRegionInfoOrBuilderList();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
         int index);
 
     /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    boolean hasMergedRegionInfo();
-    /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo();
-    /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder();
-
-    /**
-     * <code>optional bool forcible = 4 [default = false];</code>
+     * <code>optional bool forcible = 4;</code>
      */
     boolean hasForcible();
     /**
-     * <code>optional bool forcible = 4 [default = false];</code>
+     * <code>optional bool forcible = 4;</code>
      */
     boolean getForcible();
   }
   /**
-   * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
+   * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
    */
-  public  static final class MergeTableRegionsStateData extends
+  public  static final class DispatchMergingRegionsStateData extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsStateData)
-      MergeTableRegionsStateDataOrBuilder {
-    // Use MergeTableRegionsStateData.newBuilder() to construct.
-    private MergeTableRegionsStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsStateData)
+      DispatchMergingRegionsStateDataOrBuilder {
+    // Use DispatchMergingRegionsStateData.newBuilder() to construct.
+    private DispatchMergingRegionsStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private MergeTableRegionsStateData() {
+    private DispatchMergingRegionsStateData() {
       regionInfo_ = java.util.Collections.emptyList();
       forcible_ = false;
     }
@@ -19976,7 +20282,7 @@ public final class MasterProcedureProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private MergeTableRegionsStateData(
+    private DispatchMergingRegionsStateData(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -20013,27 +20319,27 @@ public final class MasterProcedureProtos {
               break;
             }
             case 18: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
-                mutable_bitField0_ |= 0x00000002;
-              }
-              regionInfo_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
-              break;
-            }
-            case 26: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
               if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = mergedRegionInfo_.toBuilder();
+                subBuilder = tableName_.toBuilder();
               }
-              mergedRegionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(mergedRegionInfo_);
-                mergedRegionInfo_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
               }
               bitField0_ |= 0x00000002;
               break;
             }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              regionInfo_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              break;
+            }
             case 32: {
               bitField0_ |= 0x00000004;
               forcible_ = input.readBool();
@@ -20047,7 +20353,7 @@ public final class MasterProcedureProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
           regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
         }
         this.unknownFields = unknownFields.build();
@@ -20056,14 +20362,14 @@ public final class MasterProcedureProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
     }
 
     private int bitField0_;
@@ -20088,72 +20394,72 @@ public final class MasterProcedureProtos {
       return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
     }
 
-    public static final int REGION_INFO_FIELD_NUMBER = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+    /**
+     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+
+    public static final int REGION_INFO_FIELD_NUMBER = 3;
     private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
       return regionInfo_;
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
         getRegionInfoOrBuilderList() {
       return regionInfo_;
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     public int getRegionInfoCount() {
       return regionInfo_.size();
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
       return regionInfo_.get(index);
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
         int index) {
       return regionInfo_.get(index);
     }
 
-    public static final int MERGED_REGION_INFO_FIELD_NUMBER = 3;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo mergedRegionInfo_;
-    /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    public boolean hasMergedRegionInfo() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo() {
-      return mergedRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
-    }
-    /**
-     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder() {
-      return mergedRegionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedRegionInfo_;
-    }
-
     public static final int FORCIBLE_FIELD_NUMBER = 4;
     private boolean forcible_;
     /**
-     * <code>optional bool forcible = 4 [default = false];</code>
+     * <code>optional bool forcible = 4;</code>
      */
     public boolean hasForcible() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional bool forcible = 4 [default = false];</code>
+     * <code>optional bool forcible = 4;</code>
      */
     public boolean getForcible() {
       return forcible_;
@@ -20169,7 +20475,7 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasMergedRegionInfo()) {
+      if (!hasTableName()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -20177,16 +20483,16 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       for (int i = 0; i < getRegionInfoCount(); i++) {
         if (!getRegionInfo(i).isInitialized()) {
           memoizedIsInitialized = 0;
           return false;
         }
       }
-      if (!getMergedRegionInfo().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -20196,11 +20502,11 @@ public final class MasterProcedureProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
       }
-      for (int i = 0; i < regionInfo_.size(); i++) {
-        output.writeMessage(2, regionInfo_.get(i));
-      }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(3, getMergedRegionInfo());
+        output.writeMessage(2, getTableName());
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        output.writeMessage(3, regionInfo_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeBool(4, forcible_);
@@ -20217,13 +20523,13 @@ public final class MasterProcedureProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
-      for (int i = 0; i < regionInfo_.size(); i++) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, regionInfo_.get(i));
+          .computeMessageSize(2, getTableName());
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+      for (int i = 0; i < regionInfo_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, getMergedRegionInfo());
+          .computeMessageSize(3, regionInfo_.get(i));
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
@@ -20240,10 +20546,10 @@ public final class MasterProcedureProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) obj;
 
       boolean result = true;
       result = result && (hasUserInfo() == other.hasUserInfo());
@@ -20251,13 +20557,13 @@ public final class MasterProcedureProtos {
         result = result && getUserInfo()
             .equals(other.getUserInfo());
       }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
       result = result && getRegionInfoList()
           .equals(other.getRegionInfoList());
-      result = result && (hasMergedRegionInfo() == other.hasMergedRegionInfo());
-      if (hasMergedRegionInfo()) {
-        result = result && getMergedRegionInfo()
-            .equals(other.getMergedRegionInfo());
-      }
       result = result && (hasForcible() == other.hasForcible());
       if (hasForcible()) {
         result = result && (getForcible()
@@ -20278,14 +20584,14 @@ public final class MasterProcedureProtos {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
       }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
       if (getRegionInfoCount() > 0) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getRegionInfoList().hashCode();
       }
-      if (hasMergedRegionInfo()) {
-        hash = (37 * hash) + MERGED_REGION_INFO_FIELD_NUMBER;
-        hash = (53 * hash) + getMergedRegionInfo().hashCode();
-      }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
@@ -20296,58 +20602,58 @@ public final class MasterProcedureProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(byte[] data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         byte[] data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -20359,7 +20665,7 @@ public final class MasterProcedureProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -20374,25 +20680,25 @@ public final class MasterProcedureProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
+     * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
      */
     public static final class Builder extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsStateData)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateDataOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsStateData)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateDataOrBuilder {
       public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
       }
 
       protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -20406,8 +20712,8 @@ public final class MasterProcedureProtos {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
           getRegionInfoFieldBuilder();
-          getMergedRegionInfoFieldBuilder();
         }
       }
       public Builder clear() {
@@ -20418,18 +20724,18 @@ public final class MasterProcedureProtos {
           userInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
         if (regionInfoBuilder_ == null) {
           regionInfo_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000004);
         } else {
           regionInfoBuilder_.clear();
         }
-        if (mergedRegionInfoBuilder_ == null) {
-          mergedRegionInfo_ = null;
-        } else {
-          mergedRegionInfoBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000004);
         forcible_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
@@ -20437,23 +20743,23 @@ public final class MasterProcedureProtos {
 
       public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -20464,23 +20770,23 @@ public final class MasterProcedureProtos {
         } else {
           result.userInfo_ = userInfoBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
         if (regionInfoBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
             regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
-            bitField0_ = (bitField0_ & ~0x00000002);
+            bitField0_ = (bitField0_ & ~0x00000004);
           }
           result.regionInfo_ = regionInfo_;
         } else {
           result.regionInfo_ = regionInfoBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        if (mergedRegionInfoBuilder_ == null) {
-          result.mergedRegionInfo_ = mergedRegionInfo_;
-        } else {
-          result.mergedRegionInfo_ = mergedRegionInfoBuilder_.build();
-        }
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
           to_bitField0_ |= 0x00000004;
         }
@@ -20517,24 +20823,27 @@ public final class MasterProcedureProtos {
         return (Builder) super.addRepeatedField(field, value);
       }
       public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.getDefaultInstance()) return this;
         if (other.hasUserInfo()) {
           mergeUserInfo(other.getUserInfo());
         }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
         if (regionInfoBuilder_ == null) {
           if (!other.regionInfo_.isEmpty()) {
             if (regionInfo_.isEmpty()) {
               regionInfo_ = other.regionInfo_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000004);
             } else {
               ensureRegionInfoIsMutable();
               regionInfo_.addAll(other.regionInfo_);
@@ -20547,7 +20856,7 @@ public final class MasterProcedureProtos {
               regionInfoBuilder_.dispose();
               regionInfoBuilder_ = null;
               regionInfo_ = other.regionInfo_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000004);
               regionInfoBuilder_ = 
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
@@ -20556,9 +20865,6 @@ public final class MasterProcedureProtos {
             }
           }
         }
-        if (other.hasMergedRegionInfo()) {
-          mergeMergedRegionInfo(other.getMergedRegionInfo());
-        }
         if (other.hasForcible()) {
           setForcible(other.getForcible());
         }
@@ -20571,20 +20877,20 @@ public final class MasterProcedureProtos {
         if (!hasUserInfo()) {
           return false;
         }
-        if (!hasMergedRegionInfo()) {
+        if (!hasTableName()) {
           return false;
         }
         if (!getUserInfo().isInitialized()) {
           return false;
         }
+        if (!getTableName().isInitialized()) {
+          return false;
+        }
         for (int i = 0; i < getRegionInfoCount(); i++) {
           if (!getRegionInfo(i).isInitialized()) {
             return false;
           }
         }
-        if (!getMergedRegionInfo().isInitialized()) {
-          return false;
-        }
         return true;
       }
 
@@ -20592,11 +20898,11 @@ public final class MasterProcedureProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -20725,12 +21031,130 @@ public final class MasterProcedureProtos {
         return userInfoBuilder_;
       }
 
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != null &&
+              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.TableName table_name = 2;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  getTableName(),
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
       private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
         java.util.Collections.emptyList();
       private void ensureRegionInfoIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
           regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
-          bitField0_ |= 0x00000002;
+          bitField0_ |= 0x00000004;
          }
       }
 
@@ -20738,7 +21162,7 @@ public final class MasterProcedureProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
 
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
         if (regionInfoBuilder_ == null) {
@@ -20748,7 +21172,7 @@ public final class MasterProcedureProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public int getRegionInfoCount() {
         if (regionInfoBuilder_ == null) {
@@ -20758,7 +21182,7 @@ public final class MasterProcedureProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
         if (regionInfoBuilder_ == null) {
@@ -20768,7 +21192,7 @@ public final class MasterProcedureProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder setRegionInfo(
           int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
@@ -20785,7 +21209,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder setRegionInfo(
           int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
@@ -20799,7 +21223,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder addRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
         if (regionInfoBuilder_ == null) {
@@ -20815,7 +21239,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder addRegionInfo(
           int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
@@ -20832,7 +21256,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder addRegionInfo(
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
@@ -20846,7 +21270,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder addRegionInfo(
           int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
@@ -20860,7 +21284,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder addAllRegionInfo(
           java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> values) {
@@ -20875,12 +21299,12 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder clearRegionInfo() {
         if (regionInfoBuilder_ == null) {
           regionInfo_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000004);
           onChanged();
         } else {
           regionInfoBuilder_.clear();
@@ -20888,7 +21312,7 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public Builder removeRegionInfo(int index) {
         if (regionInfoBuilder_ == null) {
@@ -20901,14 +21325,14 @@ public final class MasterProcedureProtos {
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
           int index) {
         return getRegionInfoFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
           int index) {
@@ -20918,7 +21342,7 @@ public final class MasterProcedureProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
            getRegionInfoOrBuilderList() {
@@ -20929,14 +21353,14 @@ public final class MasterProcedureProtos {
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
         return getRegionInfoFieldBuilder().addBuilder(
             org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
           int index) {
@@ -20944,7 +21368,7 @@ public final class MasterProcedureProtos {
             index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
       }
       /**
-       * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
+       * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
        */
       public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder> 
            getRegionInfoBuilderList() {
@@ -20957,7 +21381,7 @@ public final class MasterProcedureProtos {
           regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generate

<TRUNCATED>

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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
index 8f368e9..b8712b7 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/RegionServerStatusProtos.java
@@ -8822,1348 +8822,6 @@ public final class RegionServerStatusProtos {
 
   }
 
-  public interface SplitTableRegionRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.SplitTableRegionRequest)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    boolean hasRegionInfo();
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo();
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder();
-
-    /**
-     * <code>required bytes split_row = 2;</code>
-     */
-    boolean hasSplitRow();
-    /**
-     * <code>required bytes split_row = 2;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitRow();
-
-    /**
-     * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-     */
-    boolean hasNonceGroup();
-    /**
-     * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-     */
-    long getNonceGroup();
-
-    /**
-     * <code>optional uint64 nonce = 4 [default = 0];</code>
-     */
-    boolean hasNonce();
-    /**
-     * <code>optional uint64 nonce = 4 [default = 0];</code>
-     */
-    long getNonce();
-  }
-  /**
-   * <pre>
-   **
-   * Splits the specified region.
-   * </pre>
-   *
-   * Protobuf type {@code hbase.pb.SplitTableRegionRequest}
-   */
-  public  static final class SplitTableRegionRequest extends
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.SplitTableRegionRequest)
-      SplitTableRegionRequestOrBuilder {
-    // Use SplitTableRegionRequest.newBuilder() to construct.
-    private SplitTableRegionRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private SplitTableRegionRequest() {
-      splitRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
-      nonceGroup_ = 0L;
-      nonce_ = 0L;
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private SplitTableRegionRequest(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = regionInfo_.toBuilder();
-              }
-              regionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(regionInfo_);
-                regionInfo_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              splitRow_ = input.readBytes();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              nonceGroup_ = input.readUInt64();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              nonce_ = input.readUInt64();
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionRequest_descriptor;
-    }
-
-    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionRequest_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int REGION_INFO_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_;
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    public boolean hasRegionInfo() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
-      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
-    }
-    /**
-     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
-      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
-    }
-
-    public static final int SPLIT_ROW_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitRow_;
-    /**
-     * <code>required bytes split_row = 2;</code>
-     */
-    public boolean hasSplitRow() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>required bytes split_row = 2;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitRow() {
-      return splitRow_;
-    }
-
-    public static final int NONCE_GROUP_FIELD_NUMBER = 3;
-    private long nonceGroup_;
-    /**
-     * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-     */
-    public boolean hasNonceGroup() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-     */
-    public long getNonceGroup() {
-      return nonceGroup_;
-    }
-
-    public static final int NONCE_FIELD_NUMBER = 4;
-    private long nonce_;
-    /**
-     * <code>optional uint64 nonce = 4 [default = 0];</code>
-     */
-    public boolean hasNonce() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional uint64 nonce = 4 [default = 0];</code>
-     */
-    public long getNonce() {
-      return nonce_;
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasRegionInfo()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!hasSplitRow()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getRegionInfo().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, getRegionInfo());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, splitRow_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeUInt64(3, nonceGroup_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeUInt64(4, nonce_);
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, getRegionInfo());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, splitRow_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(3, nonceGroup_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(4, nonce_);
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest) obj;
-
-      boolean result = true;
-      result = result && (hasRegionInfo() == other.hasRegionInfo());
-      if (hasRegionInfo()) {
-        result = result && getRegionInfo()
-            .equals(other.getRegionInfo());
-      }
-      result = result && (hasSplitRow() == other.hasSplitRow());
-      if (hasSplitRow()) {
-        result = result && getSplitRow()
-            .equals(other.getSplitRow());
-      }
-      result = result && (hasNonceGroup() == other.hasNonceGroup());
-      if (hasNonceGroup()) {
-        result = result && (getNonceGroup()
-            == other.getNonceGroup());
-      }
-      result = result && (hasNonce() == other.hasNonce());
-      if (hasNonce()) {
-        result = result && (getNonce()
-            == other.getNonce());
-      }
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasRegionInfo()) {
-        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionInfo().hashCode();
-      }
-      if (hasSplitRow()) {
-        hash = (37 * hash) + SPLIT_ROW_FIELD_NUMBER;
-        hash = (53 * hash) + getSplitRow().hashCode();
-      }
-      if (hasNonceGroup()) {
-        hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getNonceGroup());
-      }
-      if (hasNonce()) {
-        hash = (37 * hash) + NONCE_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getNonce());
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(byte[] data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        byte[] data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseDelimitedFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * <pre>
-     **
-     * Splits the specified region.
-     * </pre>
-     *
-     * Protobuf type {@code hbase.pb.SplitTableRegionRequest}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.SplitTableRegionRequest)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequestOrBuilder {
-      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionRequest_descriptor;
-      }
-
-      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionRequest_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-          getRegionInfoFieldBuilder();
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        if (regionInfoBuilder_ == null) {
-          regionInfo_ = null;
-        } else {
-          regionInfoBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        splitRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        nonceGroup_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        nonce_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        return this;
-      }
-
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionRequest_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (regionInfoBuilder_ == null) {
-          result.regionInfo_ = regionInfo_;
-        } else {
-          result.regionInfo_ = regionInfoBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.splitRow_ = splitRow_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.nonceGroup_ = nonceGroup_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.nonce_ = nonce_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.getDefaultInstance()) return this;
-        if (other.hasRegionInfo()) {
-          mergeRegionInfo(other.getRegionInfo());
-        }
-        if (other.hasSplitRow()) {
-          setSplitRow(other.getSplitRow());
-        }
-        if (other.hasNonceGroup()) {
-          setNonceGroup(other.getNonceGroup());
-        }
-        if (other.hasNonce()) {
-          setNonce(other.getNonce());
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasRegionInfo()) {
-          return false;
-        }
-        if (!hasSplitRow()) {
-          return false;
-        }
-        if (!getRegionInfo().isInitialized()) {
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest) e.getUnfinishedMessage();
-          throw e.unwrapIOException();
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public boolean hasRegionInfo() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
-        if (regionInfoBuilder_ == null) {
-          return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
-        } else {
-          return regionInfoBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public Builder setRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
-        if (regionInfoBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          regionInfo_ = value;
-          onChanged();
-        } else {
-          regionInfoBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public Builder setRegionInfo(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
-        if (regionInfoBuilder_ == null) {
-          regionInfo_ = builderForValue.build();
-          onChanged();
-        } else {
-          regionInfoBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public Builder mergeRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
-        if (regionInfoBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              regionInfo_ != null &&
-              regionInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
-            regionInfo_ =
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(regionInfo_).mergeFrom(value).buildPartial();
-          } else {
-            regionInfo_ = value;
-          }
-          onChanged();
-        } else {
-          regionInfoBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public Builder clearRegionInfo() {
-        if (regionInfoBuilder_ == null) {
-          regionInfo_ = null;
-          onChanged();
-        } else {
-          regionInfoBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getRegionInfoFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
-        if (regionInfoBuilder_ != null) {
-          return regionInfoBuilder_.getMessageOrBuilder();
-        } else {
-          return regionInfo_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
-        }
-      }
-      /**
-       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
-       */
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
-          getRegionInfoFieldBuilder() {
-        if (regionInfoBuilder_ == null) {
-          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
-                  getRegionInfo(),
-                  getParentForChildren(),
-                  isClean());
-          regionInfo_ = null;
-        }
-        return regionInfoBuilder_;
-      }
-
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString splitRow_ = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>required bytes split_row = 2;</code>
-       */
-      public boolean hasSplitRow() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>required bytes split_row = 2;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getSplitRow() {
-        return splitRow_;
-      }
-      /**
-       * <code>required bytes split_row = 2;</code>
-       */
-      public Builder setSplitRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        splitRow_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bytes split_row = 2;</code>
-       */
-      public Builder clearSplitRow() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        splitRow_ = getDefaultInstance().getSplitRow();
-        onChanged();
-        return this;
-      }
-
-      private long nonceGroup_ ;
-      /**
-       * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-       */
-      public boolean hasNonceGroup() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-       */
-      public long getNonceGroup() {
-        return nonceGroup_;
-      }
-      /**
-       * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-       */
-      public Builder setNonceGroup(long value) {
-        bitField0_ |= 0x00000004;
-        nonceGroup_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 nonce_group = 3 [default = 0];</code>
-       */
-      public Builder clearNonceGroup() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        nonceGroup_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      private long nonce_ ;
-      /**
-       * <code>optional uint64 nonce = 4 [default = 0];</code>
-       */
-      public boolean hasNonce() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional uint64 nonce = 4 [default = 0];</code>
-       */
-      public long getNonce() {
-        return nonce_;
-      }
-      /**
-       * <code>optional uint64 nonce = 4 [default = 0];</code>
-       */
-      public Builder setNonce(long value) {
-        bitField0_ |= 0x00000008;
-        nonce_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 nonce = 4 [default = 0];</code>
-       */
-      public Builder clearNonce() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        nonce_ = 0L;
-        onChanged();
-        return this;
-      }
-      public final Builder setUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.setUnknownFields(unknownFields);
-      }
-
-      public final Builder mergeUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.mergeUnknownFields(unknownFields);
-      }
-
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SplitTableRegionRequest)
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SplitTableRegionRequest)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest DEFAULT_INSTANCE;
-    static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest();
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest getDefaultInstance() {
-      return DEFAULT_INSTANCE;
-    }
-
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionRequest>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SplitTableRegionRequest>() {
-      public SplitTableRegionRequest parsePartialFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new SplitTableRegionRequest(input, extensionRegistry);
-      }
-    };
-
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionRequest> parser() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionRequest> getParserForType() {
-      return PARSER;
-    }
-
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest getDefaultInstanceForType() {
-      return DEFAULT_INSTANCE;
-    }
-
-  }
-
-  public interface SplitTableRegionResponseOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.SplitTableRegionResponse)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    boolean hasProcId();
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    long getProcId();
-  }
-  /**
-   * Protobuf type {@code hbase.pb.SplitTableRegionResponse}
-   */
-  public  static final class SplitTableRegionResponse extends
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.SplitTableRegionResponse)
-      SplitTableRegionResponseOrBuilder {
-    // Use SplitTableRegionResponse.newBuilder() to construct.
-    private SplitTableRegionResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private SplitTableRegionResponse() {
-      procId_ = 0L;
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private SplitTableRegionResponse(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              procId_ = input.readUInt64();
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionResponse_descriptor;
-    }
-
-    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionResponse_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int PROC_ID_FIELD_NUMBER = 1;
-    private long procId_;
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    public boolean hasProcId() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional uint64 proc_id = 1;</code>
-     */
-    public long getProcId() {
-      return procId_;
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt64(1, procId_);
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(1, procId_);
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse) obj;
-
-      boolean result = true;
-      result = result && (hasProcId() == other.hasProcId());
-      if (hasProcId()) {
-        result = result && (getProcId()
-            == other.getProcId());
-      }
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasProcId()) {
-        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getProcId());
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(byte[] data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        byte[] data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseDelimitedFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.SplitTableRegionResponse}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.SplitTableRegionResponse)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponseOrBuilder {
-      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionResponse_descriptor;
-      }
-
-      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionResponse_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        procId_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.internal_static_hbase_pb_SplitTableRegionResponse_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.procId_ = procId_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance()) return this;
-        if (other.hasProcId()) {
-          setProcId(other.getProcId());
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse) e.getUnfinishedMessage();
-          throw e.unwrapIOException();
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private long procId_ ;
-      /**
-       * <code>optional uint64 proc_id = 1;</code>
-       */
-      public boolean hasProcId() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional uint64 proc_id = 1;</code>
-       */
-      public long getProcId() {
-        return procId_;
-      }
-      /**
-       * <code>optional uint64 proc_id = 1;</code>
-       */
-      public Builder setProcId(long value) {
-        bitField0_ |= 0x00000001;
-        procId_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional uint64 proc_id = 1;</code>
-       */
-      public Builder clearProcId() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        procId_ = 0L;
-        onChanged();
-        return this;
-      }
-      public final Builder setUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.setUnknownFields(unknownFields);
-      }
-
-      public final Builder mergeUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.mergeUnknownFields(unknownFields);
-      }
-
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.SplitTableRegionResponse)
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.SplitTableRegionResponse)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse DEFAULT_INSTANCE;
-    static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse();
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse getDefaultInstance() {
-      return DEFAULT_INSTANCE;
-    }
-
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionResponse>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<SplitTableRegionResponse>() {
-      public SplitTableRegionResponse parsePartialFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new SplitTableRegionResponse(input, extensionRegistry);
-      }
-    };
-
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionResponse> parser() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<SplitTableRegionResponse> getParserForType() {
-      return PARSER;
-    }
-
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse getDefaultInstanceForType() {
-      return DEFAULT_INSTANCE;
-    }
-
-  }
-
   /**
    * Protobuf service {@code hbase.pb.RegionServerStatusService}
    */
@@ -10239,32 +8897,6 @@ public final class RegionServerStatusProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse> done);
 
-      /**
-       * <pre>
-       **
-       * Split region
-       * </pre>
-       *
-       * <code>rpc SplitRegion(.hbase.pb.SplitTableRegionRequest) returns (.hbase.pb.SplitTableRegionResponse);</code>
-       */
-      public abstract void splitRegion(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse> done);
-
-      /**
-       * <pre>
-       **
-       * Get procedure result
-       * </pre>
-       *
-       * <code>rpc getProcedureResult(.hbase.pb.GetProcedureResultRequest) returns (.hbase.pb.GetProcedureResultResponse);</code>
-       */
-      public abstract void getProcedureResult(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse> done);
-
     }
 
     public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Service newReflectiveService(
@@ -10310,22 +8942,6 @@ public final class RegionServerStatusProtos {
           impl.reportRegionStateTransition(controller, request, done);
         }
 
-        @java.lang.Override
-        public  void splitRegion(
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse> done) {
-          impl.splitRegion(controller, request, done);
-        }
-
-        @java.lang.Override
-        public  void getProcedureResult(
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse> done) {
-          impl.getProcedureResult(controller, request, done);
-        }
-
       };
     }
 
@@ -10358,10 +8974,6 @@ public final class RegionServerStatusProtos {
               return impl.getLastFlushedSequenceId(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest)request);
             case 4:
               return impl.reportRegionStateTransition(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest)request);
-            case 5:
-              return impl.splitRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest)request);
-            case 6:
-              return impl.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -10386,10 +8998,6 @@ public final class RegionServerStatusProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest.getDefaultInstance();
             case 4:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest.getDefaultInstance();
-            case 5:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.getDefaultInstance();
-            case 6:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -10414,10 +9022,6 @@ public final class RegionServerStatusProtos {
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse.getDefaultInstance();
             case 4:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance();
-            case 5:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance();
-            case 6:
-              return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -10493,32 +9097,6 @@ public final class RegionServerStatusProtos {
         org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse> done);
 
-    /**
-     * <pre>
-     **
-     * Split region
-     * </pre>
-     *
-     * <code>rpc SplitRegion(.hbase.pb.SplitTableRegionRequest) returns (.hbase.pb.SplitTableRegionResponse);</code>
-     */
-    public abstract void splitRegion(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-        org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse> done);
-
-    /**
-     * <pre>
-     **
-     * Get procedure result
-     * </pre>
-     *
-     * <code>rpc getProcedureResult(.hbase.pb.GetProcedureResultRequest) returns (.hbase.pb.GetProcedureResultResponse);</code>
-     */
-    public abstract void getProcedureResult(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse> done);
-
     public static final
         org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -10566,16 +9144,6 @@ public final class RegionServerStatusProtos {
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse>specializeCallback(
               done));
           return;
-        case 5:
-          this.splitRegion(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest)request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse>specializeCallback(
-              done));
-          return;
-        case 6:
-          this.getProcedureResult(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest)request,
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse>specializeCallback(
-              done));
-          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -10600,10 +9168,6 @@ public final class RegionServerStatusProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest.getDefaultInstance();
         case 4:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest.getDefaultInstance();
-        case 5:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest.getDefaultInstance();
-        case 6:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -10628,10 +9192,6 @@ public final class RegionServerStatusProtos {
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse.getDefaultInstance();
         case 4:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance();
-        case 5:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance();
-        case 6:
-          return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -10727,36 +9287,6 @@ public final class RegionServerStatusProtos {
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.class,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance()));
       }
-
-      public  void splitRegion(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(5),
-          controller,
-          request,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance(),
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.class,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance()));
-      }
-
-      public  void getProcedureResult(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(6),
-          controller,
-          request,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance(),
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.class,
-            org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance()));
-      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -10789,16 +9319,6 @@ public final class RegionServerStatusProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse splitRegion(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse getProcedureResult(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -10867,30 +9387,6 @@ public final class RegionServerStatusProtos {
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse.getDefaultInstance());
       }
 
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse splitRegion(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionRequest request)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
-        return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(5),
-          controller,
-          request,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.SplitTableRegionResponse.getDefaultInstance());
-      }
-
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse getProcedureResult(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest request)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
-        return (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(6),
-          controller,
-          request,
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance());
-      }
-
     }
 
     // @@protoc_insertion_point(class_scope:hbase.pb.RegionServerStatusService)
@@ -10951,16 +9447,6 @@ public final class RegionServerStatusProtos {
   private static final 
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ReportRegionStateTransitionResponse_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SplitTableRegionRequest_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SplitTableRegionRequest_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SplitTableRegionResponse_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SplitTableRegionResponse_fieldAccessorTable;
 
   public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -10971,63 +9457,54 @@ public final class RegionServerStatusProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\030RegionServerStatus.proto\022\010hbase.pb\032\013HB" +
-      "ase.proto\032\014Master.proto\032\023ClusterStatus.p" +
-      "roto\"\205\001\n\032RegionServerStartupRequest\022\014\n\004p" +
-      "ort\030\001 \002(\r\022\031\n\021server_start_code\030\002 \002(\004\022\033\n\023" +
-      "server_current_time\030\003 \002(\004\022!\n\031use_this_ho" +
-      "stname_instead\030\004 \001(\t\"L\n\033RegionServerStar" +
-      "tupResponse\022-\n\013map_entries\030\001 \003(\0132\030.hbase" +
-      ".pb.NameStringPair\"e\n\031RegionServerReport" +
-      "Request\022$\n\006server\030\001 \002(\0132\024.hbase.pb.Serve" +
-      "rName\022\"\n\004load\030\002 \001(\0132\024.hbase.pb.ServerLoa",
-      "d\"\034\n\032RegionServerReportResponse\"X\n\031Repor" +
-      "tRSFatalErrorRequest\022$\n\006server\030\001 \002(\0132\024.h" +
-      "base.pb.ServerName\022\025\n\rerror_message\030\002 \002(" +
-      "\t\"\034\n\032ReportRSFatalErrorResponse\"6\n\037GetLa" +
-      "stFlushedSequenceIdRequest\022\023\n\013region_nam" +
-      "e\030\001 \002(\014\"\207\001\n GetLastFlushedSequenceIdResp" +
-      "onse\022 \n\030last_flushed_sequence_id\030\001 \002(\004\022A" +
-      "\n\036store_last_flushed_sequence_id\030\002 \003(\0132\031" +
-      ".hbase.pb.StoreSequenceId\"\344\002\n\025RegionStat" +
-      "eTransition\022G\n\017transition_code\030\001 \002(\0162..h",
-      "base.pb.RegionStateTransition.Transition" +
-      "Code\022)\n\013region_info\030\002 \003(\0132\024.hbase.pb.Reg" +
-      "ionInfo\022\024\n\014open_seq_num\030\003 \001(\004\"\300\001\n\016Transi" +
-      "tionCode\022\n\n\006OPENED\020\000\022\017\n\013FAILED_OPEN\020\001\022\n\n" +
-      "\006CLOSED\020\002\022\022\n\016READY_TO_SPLIT\020\003\022\022\n\016READY_T" +
-      "O_MERGE\020\004\022\016\n\nSPLIT_PONR\020\005\022\016\n\nMERGE_PONR\020" +
-      "\006\022\t\n\005SPLIT\020\007\022\n\n\006MERGED\020\010\022\022\n\016SPLIT_REVERT" +
-      "ED\020\t\022\022\n\016MERGE_REVERTED\020\n\"\177\n\"ReportRegion" +
-      "StateTransitionRequest\022$\n\006server\030\001 \002(\0132\024" +
-      ".hbase.pb.ServerName\0223\n\ntransition\030\002 \003(\013",
-      "2\037.hbase.pb.RegionStateTransition\"<\n#Rep" +
-      "ortRegionStateTransitionResponse\022\025\n\rerro" +
-      "r_message\030\001 \001(\t\"\201\001\n\027SplitTableRegionRequ" +
-      "est\022)\n\013region_info\030\001 \002(\0132\024.hbase.pb.Regi" +
-      "onInfo\022\021\n\tsplit_row\030\002 \002(\014\022\026\n\013nonce_group" +
-      "\030\003 \001(\004:\0010\022\020\n\005nonce\030\004 \001(\004:\0010\"+\n\030SplitTabl" +
-      "eRegionResponse\022\017\n\007proc_id\030\001 \001(\0042\347\005\n\031Reg" +
-      "ionServerStatusService\022b\n\023RegionServerSt" +
-      "artup\022$.hbase.pb.RegionServerStartupRequ" +
-      "est\032%.hbase.pb.RegionServerStartupRespon",
-      "se\022_\n\022RegionServerReport\022#.hbase.pb.Regi" +
-      "onServerReportRequest\032$.hbase.pb.RegionS" +
-      "erverReportResponse\022_\n\022ReportRSFatalErro" +
-      "r\022#.hbase.pb.ReportRSFatalErrorRequest\032$" +
-      ".hbase.pb.ReportRSFatalErrorResponse\022q\n\030" +
-      "GetLastFlushedSequenceId\022).hbase.pb.GetL" +
-      "astFlushedSequenceIdRequest\032*.hbase.pb.G" +
-      "etLastFlushedSequenceIdResponse\022z\n\033Repor" +
-      "tRegionStateTransition\022,.hbase.pb.Report" +
-      "RegionStateTransitionRequest\032-.hbase.pb.",
-      "ReportRegionStateTransitionResponse\022T\n\013S" +
-      "plitRegion\022!.hbase.pb.SplitTableRegionRe" +
-      "quest\032\".hbase.pb.SplitTableRegionRespons" +
-      "e\022_\n\022getProcedureResult\022#.hbase.pb.GetPr" +
-      "ocedureResultRequest\032$.hbase.pb.GetProce" +
-      "dureResultResponseBU\n1org.apache.hadoop." +
-      "hbase.shaded.protobuf.generatedB\030RegionS" +
-      "erverStatusProtosH\001\210\001\001\240\001\001"
+      "ase.proto\032\023ClusterStatus.proto\"\205\001\n\032Regio" +
+      "nServerStartupRequest\022\014\n\004port\030\001 \002(\r\022\031\n\021s" +
+      "erver_start_code\030\002 \002(\004\022\033\n\023server_current" +
+      "_time\030\003 \002(\004\022!\n\031use_this_hostname_instead" +
+      "\030\004 \001(\t\"L\n\033RegionServerStartupResponse\022-\n" +
+      "\013map_entries\030\001 \003(\0132\030.hbase.pb.NameString" +
+      "Pair\"e\n\031RegionServerReportRequest\022$\n\006ser" +
+      "ver\030\001 \002(\0132\024.hbase.pb.ServerName\022\"\n\004load\030" +
+      "\002 \001(\0132\024.hbase.pb.ServerLoad\"\034\n\032RegionSer",
+      "verReportResponse\"X\n\031ReportRSFatalErrorR" +
+      "equest\022$\n\006server\030\001 \002(\0132\024.hbase.pb.Server" +
+      "Name\022\025\n\rerror_message\030\002 \002(\t\"\034\n\032ReportRSF" +
+      "atalErrorResponse\"6\n\037GetLastFlushedSeque" +
+      "nceIdRequest\022\023\n\013region_name\030\001 \002(\014\"\207\001\n Ge" +
+      "tLastFlushedSequenceIdResponse\022 \n\030last_f" +
+      "lushed_sequence_id\030\001 \002(\004\022A\n\036store_last_f" +
+      "lushed_sequence_id\030\002 \003(\0132\031.hbase.pb.Stor" +
+      "eSequenceId\"\344\002\n\025RegionStateTransition\022G\n" +
+      "\017transition_code\030\001 \002(\0162..hbase.pb.Region",
+      "StateTransition.TransitionCode\022)\n\013region" +
+      "_info\030\002 \003(\0132\024.hbase.pb.RegionInfo\022\024\n\014ope" +
+      "n_seq_num\030\003 \001(\004\"\300\001\n\016TransitionCode\022\n\n\006OP" +
+      "ENED\020\000\022\017\n\013FAILED_OPEN\020\001\022\n\n\006CLOSED\020\002\022\022\n\016R" +
+      "EADY_TO_SPLIT\020\003\022\022\n\016READY_TO_MERGE\020\004\022\016\n\nS" +
+      "PLIT_PONR\020\005\022\016\n\nMERGE_PONR\020\006\022\t\n\005SPLIT\020\007\022\n" +
+      "\n\006MERGED\020\010\022\022\n\016SPLIT_REVERTED\020\t\022\022\n\016MERGE_" +
+      "REVERTED\020\n\"\177\n\"ReportRegionStateTransitio" +
+      "nRequest\022$\n\006server\030\001 \002(\0132\024.hbase.pb.Serv" +
+      "erName\0223\n\ntransition\030\002 \003(\0132\037.hbase.pb.Re",
+      "gionStateTransition\"<\n#ReportRegionState" +
+      "TransitionResponse\022\025\n\rerror_message\030\001 \001(" +
+      "\t2\260\004\n\031RegionServerStatusService\022b\n\023Regio" +
+      "nServerStartup\022$.hbase.pb.RegionServerSt" +
+      "artupRequest\032%.hbase.pb.RegionServerStar" +
+      "tupResponse\022_\n\022RegionServerReport\022#.hbas" +
+      "e.pb.RegionServerReportRequest\032$.hbase.p" +
+      "b.RegionServerReportResponse\022_\n\022ReportRS" +
+      "FatalError\022#.hbase.pb.ReportRSFatalError" +
+      "Request\032$.hbase.pb.ReportRSFatalErrorRes",
+      "ponse\022q\n\030GetLastFlushedSequenceId\022).hbas" +
+      "e.pb.GetLastFlushedSequenceIdRequest\032*.h" +
+      "base.pb.GetLastFlushedSequenceIdResponse" +
+      "\022z\n\033ReportRegionStateTransition\022,.hbase." +
+      "pb.ReportRegionStateTransitionRequest\032-." +
+      "hbase.pb.ReportRegionStateTransitionResp" +
+      "onseBU\n1org.apache.hadoop.hbase.shaded.p" +
+      "rotobuf.generatedB\030RegionServerStatusPro" +
+      "tosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -11041,7 +9518,6 @@ public final class RegionServerStatusProtos {
       .internalBuildGeneratedFileFrom(descriptorData,
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor[] {
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor(),
-          org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.getDescriptor(),
           org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor(),
         }, assigner);
     internal_static_hbase_pb_RegionServerStartupRequest_descriptor =
@@ -11110,20 +9586,7 @@ public final class RegionServerStatusProtos {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_ReportRegionStateTransitionResponse_descriptor,
         new java.lang.String[] { "ErrorMessage", });
-    internal_static_hbase_pb_SplitTableRegionRequest_descriptor =
-      getDescriptor().getMessageTypes().get(11);
-    internal_static_hbase_pb_SplitTableRegionRequest_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SplitTableRegionRequest_descriptor,
-        new java.lang.String[] { "RegionInfo", "SplitRow", "NonceGroup", "Nonce", });
-    internal_static_hbase_pb_SplitTableRegionResponse_descriptor =
-      getDescriptor().getMessageTypes().get(12);
-    internal_static_hbase_pb_SplitTableRegionResponse_fieldAccessorTable = new
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
-        internal_static_hbase_pb_SplitTableRegionResponse_descriptor,
-        new java.lang.String[] { "ProcId", });
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.getDescriptor();
-    org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.getDescriptor();
     org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.getDescriptor();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 338c80b..6e851e6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -119,18 +119,6 @@ message CloseRegionResponse {
 }
 
 /**
- * Closes the specified region(s) for
- * split or merge
- */
-message CloseRegionForSplitOrMergeRequest {
-  repeated RegionSpecifier region = 1;
-}
-
-message CloseRegionForSplitOrMergeResponse {
-  required bool closed = 1;
-}
-
-/**
  * Flushes the MemStore of the specified region.
  * <p>
  * This method is synchronous.
@@ -260,6 +248,32 @@ message GetRegionLoadResponse {
   repeated RegionLoad region_loads = 1;
 }
 
+message ExecuteProceduresRequest {
+  repeated OpenRegionRequest open_region = 1;
+  repeated CloseRegionRequest close_region = 2;
+}
+
+message ExecuteProceduresResponse {
+  repeated OpenRegionResponse open_region = 1;
+  repeated CloseRegionResponse close_region = 2;
+}
+
+/**
+ * Merges the specified regions.
+ * <p>
+ * This method currently closes the regions and then merges them
+ */
+message MergeRegionsRequest {
+  required RegionSpecifier region_a = 1;
+  required RegionSpecifier region_b = 2;
+  optional bool forcible = 3 [default = false];
+  // wall clock time from master
+  optional uint64 master_system_time = 4;
+}
+
+message MergeRegionsResponse {
+}
+
 service AdminService {
   rpc GetRegionInfo(GetRegionInfoRequest)
     returns(GetRegionInfoResponse);
@@ -279,9 +293,6 @@ service AdminService {
   rpc CloseRegion(CloseRegionRequest)
     returns(CloseRegionResponse);
 
-  rpc CloseRegionForSplitOrMerge(CloseRegionForSplitOrMergeRequest)
-    returns(CloseRegionForSplitOrMergeResponse);
-
   rpc FlushRegion(FlushRegionRequest)
     returns(FlushRegionResponse);
 
@@ -314,4 +325,10 @@ service AdminService {
 
   rpc GetRegionLoad(GetRegionLoadRequest)
     returns(GetRegionLoadResponse);
+
+  rpc ExecuteProcedures(ExecuteProceduresRequest)
+    returns(ExecuteProceduresResponse);
+
+  rpc MergeRegions(MergeRegionsRequest)
+    returns(MergeRegionsResponse);
 }


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index bb6b40e..319b1c8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -5507,33 +5507,35 @@ public final class MasterProtos {
 
   }
 
-  public interface MergeTableRegionsRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsRequest)
+  public interface DispatchMergingRegionsRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> 
-        getRegionList();
+    boolean hasRegionA();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionA();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    int getRegionCount();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionAOrBuilder();
+
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-        getRegionOrBuilderList();
+    boolean hasRegionB();
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-        int index);
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionB();
+    /**
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionBOrBuilder();
 
     /**
      * <code>optional bool forcible = 3 [default = false];</code>
@@ -5565,21 +5567,20 @@ public final class MasterProtos {
   /**
    * <pre>
    **
-   * Merging the specified regions in a table.
+   * Dispatch merging the specified regions.
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
+   * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
    */
-  public  static final class MergeTableRegionsRequest extends
+  public  static final class DispatchMergingRegionsRequest extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsRequest)
-      MergeTableRegionsRequestOrBuilder {
-    // Use MergeTableRegionsRequest.newBuilder() to construct.
-    private MergeTableRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsRequest)
+      DispatchMergingRegionsRequestOrBuilder {
+    // Use DispatchMergingRegionsRequest.newBuilder() to construct.
+    private DispatchMergingRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private MergeTableRegionsRequest() {
-      region_ = java.util.Collections.emptyList();
+    private DispatchMergingRegionsRequest() {
       forcible_ = false;
       nonceGroup_ = 0L;
       nonce_ = 0L;
@@ -5590,7 +5591,7 @@ public final class MasterProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private MergeTableRegionsRequest(
+    private DispatchMergingRegionsRequest(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -5614,26 +5615,43 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-                region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
-                mutable_bitField0_ |= 0x00000001;
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = regionA_.toBuilder();
               }
-              region_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
+              regionA_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(regionA_);
+                regionA_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = regionB_.toBuilder();
+              }
+              regionB_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(regionB_);
+                regionB_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
               break;
             }
             case 24: {
-              bitField0_ |= 0x00000001;
+              bitField0_ |= 0x00000004;
               forcible_ = input.readBool();
               break;
             }
             case 32: {
-              bitField0_ |= 0x00000002;
+              bitField0_ |= 0x00000008;
               nonceGroup_ = input.readUInt64();
               break;
             }
             case 40: {
-              bitField0_ |= 0x00000004;
+              bitField0_ |= 0x00000010;
               nonce_ = input.readUInt64();
               break;
             }
@@ -5645,59 +5663,63 @@ public final class MasterProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
-          region_ = java.util.Collections.unmodifiableList(region_);
-        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.Builder.class);
     }
 
     private int bitField0_;
-    public static final int REGION_FIELD_NUMBER = 1;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
+    public static final int REGION_A_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionA_;
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
-      return region_;
+    public boolean hasRegionA() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-        getRegionOrBuilderList() {
-      return region_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionA() {
+      return regionA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionA_;
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
      */
-    public int getRegionCount() {
-      return region_.size();
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionAOrBuilder() {
+      return regionA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionA_;
     }
+
+    public static final int REGION_B_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionB_;
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
      */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
-      return region_.get(index);
+    public boolean hasRegionB() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
      */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-        int index) {
-      return region_.get(index);
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionB() {
+      return regionB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionB_;
+    }
+    /**
+     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionBOrBuilder() {
+      return regionB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionB_;
     }
 
     public static final int FORCIBLE_FIELD_NUMBER = 3;
@@ -5706,7 +5728,7 @@ public final class MasterProtos {
      * <code>optional bool forcible = 3 [default = false];</code>
      */
     public boolean hasForcible() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
      * <code>optional bool forcible = 3 [default = false];</code>
@@ -5721,7 +5743,7 @@ public final class MasterProtos {
      * <code>optional uint64 nonce_group = 4 [default = 0];</code>
      */
     public boolean hasNonceGroup() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
      * <code>optional uint64 nonce_group = 4 [default = 0];</code>
@@ -5736,7 +5758,7 @@ public final class MasterProtos {
      * <code>optional uint64 nonce = 5 [default = 0];</code>
      */
     public boolean hasNonce() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
      * <code>optional uint64 nonce = 5 [default = 0];</code>
@@ -5751,11 +5773,21 @@ public final class MasterProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      for (int i = 0; i < getRegionCount(); i++) {
-        if (!getRegion(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+      if (!hasRegionA()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRegionB()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegionA().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegionB().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
       memoizedIsInitialized = 1;
       return true;
@@ -5763,16 +5795,19 @@ public final class MasterProtos {
 
     public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
-      for (int i = 0; i < region_.size(); i++) {
-        output.writeMessage(1, region_.get(i));
-      }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(3, forcible_);
+        output.writeMessage(1, getRegionA());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(4, nonceGroup_);
+        output.writeMessage(2, getRegionB());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, forcible_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, nonceGroup_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeUInt64(5, nonce_);
       }
       unknownFields.writeTo(output);
@@ -5783,19 +5818,23 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      for (int i = 0; i < region_.size(); i++) {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, region_.get(i));
+          .computeMessageSize(1, getRegionA());
       }
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getRegionB());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, forcible_);
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonceGroup_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, nonce_);
       }
@@ -5810,14 +5849,22 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) obj;
 
       boolean result = true;
-      result = result && getRegionList()
-          .equals(other.getRegionList());
+      result = result && (hasRegionA() == other.hasRegionA());
+      if (hasRegionA()) {
+        result = result && getRegionA()
+            .equals(other.getRegionA());
+      }
+      result = result && (hasRegionB() == other.hasRegionB());
+      if (hasRegionB()) {
+        result = result && getRegionB()
+            .equals(other.getRegionB());
+      }
       result = result && (hasForcible() == other.hasForcible());
       if (hasForcible()) {
         result = result && (getForcible()
@@ -5844,9 +5891,13 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (getRegionCount() > 0) {
-        hash = (37 * hash) + REGION_FIELD_NUMBER;
-        hash = (53 * hash) + getRegionList().hashCode();
+      if (hasRegionA()) {
+        hash = (37 * hash) + REGION_A_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionA().hashCode();
+      }
+      if (hasRegionB()) {
+        hash = (37 * hash) + REGION_B_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionB().hashCode();
       }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
@@ -5868,58 +5919,58 @@ public final class MasterProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(byte[] data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         byte[] data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -5931,7 +5982,7 @@ public final class MasterProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -5948,28 +5999,28 @@ public final class MasterProtos {
     /**
      * <pre>
      **
-     * Merging the specified regions in a table.
+     * Dispatch merging the specified regions.
      * </pre>
      *
-     * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
+     * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
      */
     public static final class Builder extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsRequest)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequestOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequestOrBuilder {
       public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
       }
 
       protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -5982,66 +6033,80 @@ public final class MasterProtos {
       private void maybeForceBuilderInitialization() {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
-          getRegionFieldBuilder();
+          getRegionAFieldBuilder();
+          getRegionBFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        if (regionBuilder_ == null) {
-          region_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000001);
+        if (regionABuilder_ == null) {
+          regionA_ = null;
         } else {
-          regionBuilder_.clear();
+          regionABuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (regionBBuilder_ == null) {
+          regionB_ = null;
+        } else {
+          regionBBuilder_.clear();
         }
-        forcible_ = false;
         bitField0_ = (bitField0_ & ~0x00000002);
-        nonceGroup_ = 0L;
+        forcible_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
-        nonce_ = 0L;
+        nonceGroup_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000008);
+        nonce_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
       public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
-        if (regionBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001)) {
-            region_ = java.util.Collections.unmodifiableList(region_);
-            bitField0_ = (bitField0_ & ~0x00000001);
-          }
-          result.region_ = region_;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (regionABuilder_ == null) {
+          result.regionA_ = regionA_;
         } else {
-          result.region_ = regionBuilder_.build();
+          result.regionA_ = regionABuilder_.build();
         }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000001;
+          to_bitField0_ |= 0x00000002;
+        }
+        if (regionBBuilder_ == null) {
+          result.regionB_ = regionB_;
+        } else {
+          result.regionB_ = regionBBuilder_.build();
         }
-        result.forcible_ = forcible_;
         if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000002;
+          to_bitField0_ |= 0x00000004;
         }
-        result.nonceGroup_ = nonceGroup_;
+        result.forcible_ = forcible_;
         if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000004;
+          to_bitField0_ |= 0x00000008;
+        }
+        result.nonceGroup_ = nonceGroup_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
         }
         result.nonce_ = nonce_;
         result.bitField0_ = to_bitField0_;
@@ -6076,41 +6141,21 @@ public final class MasterProtos {
         return (Builder) super.addRepeatedField(field, value);
       }
       public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance()) return this;
-        if (regionBuilder_ == null) {
-          if (!other.region_.isEmpty()) {
-            if (region_.isEmpty()) {
-              region_ = other.region_;
-              bitField0_ = (bitField0_ & ~0x00000001);
-            } else {
-              ensureRegionIsMutable();
-              region_.addAll(other.region_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.region_.isEmpty()) {
-            if (regionBuilder_.isEmpty()) {
-              regionBuilder_.dispose();
-              regionBuilder_ = null;
-              region_ = other.region_;
-              bitField0_ = (bitField0_ & ~0x00000001);
-              regionBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getRegionFieldBuilder() : null;
-            } else {
-              regionBuilder_.addAllMessages(other.region_);
-            }
-          }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.getDefaultInstance()) return this;
+        if (other.hasRegionA()) {
+          mergeRegionA(other.getRegionA());
+        }
+        if (other.hasRegionB()) {
+          mergeRegionB(other.getRegionB());
         }
         if (other.hasForcible()) {
           setForcible(other.getForcible());
@@ -6127,10 +6172,17 @@ public final class MasterProtos {
       }
 
       public final boolean isInitialized() {
-        for (int i = 0; i < getRegionCount(); i++) {
-          if (!getRegion(i).isInitialized()) {
-            return false;
-          }
+        if (!hasRegionA()) {
+          return false;
+        }
+        if (!hasRegionB()) {
+          return false;
+        }
+        if (!getRegionA().isInitialized()) {
+          return false;
+        }
+        if (!getRegionB().isInitialized()) {
+          return false;
         }
         return true;
       }
@@ -6139,11 +6191,11 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -6154,244 +6206,240 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_ =
-        java.util.Collections.emptyList();
-      private void ensureRegionIsMutable() {
-        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
-          region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>(region_);
-          bitField0_ |= 0x00000001;
-         }
-      }
-
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
-        if (regionBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(region_);
-        } else {
-          return regionBuilder_.getMessageList();
-        }
-      }
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionA_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionABuilder_;
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public int getRegionCount() {
-        if (regionBuilder_ == null) {
-          return region_.size();
-        } else {
-          return regionBuilder_.getCount();
-        }
+      public boolean hasRegionA() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
-        if (regionBuilder_ == null) {
-          return region_.get(index);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionA() {
+        if (regionABuilder_ == null) {
+          return regionA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionA_;
         } else {
-          return regionBuilder_.getMessage(index);
+          return regionABuilder_.getMessage();
         }
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder setRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
-        if (regionBuilder_ == null) {
+      public Builder setRegionA(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionABuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          ensureRegionIsMutable();
-          region_.set(index, value);
+          regionA_ = value;
           onChanged();
         } else {
-          regionBuilder_.setMessage(index, value);
+          regionABuilder_.setMessage(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder setRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.set(index, builderForValue.build());
+      public Builder setRegionA(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+        if (regionABuilder_ == null) {
+          regionA_ = builderForValue.build();
           onChanged();
         } else {
-          regionBuilder_.setMessage(index, builderForValue.build());
+          regionABuilder_.setMessage(builderForValue.build());
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
-        if (regionBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
+      public Builder mergeRegionA(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionABuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              regionA_ != null &&
+              regionA_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) {
+            regionA_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(regionA_).mergeFrom(value).buildPartial();
+          } else {
+            regionA_ = value;
           }
-          ensureRegionIsMutable();
-          region_.add(value);
           onChanged();
         } else {
-          regionBuilder_.addMessage(value);
+          regionABuilder_.mergeFrom(value);
         }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder addRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
-        if (regionBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureRegionIsMutable();
-          region_.add(index, value);
+      public Builder clearRegionA() {
+        if (regionABuilder_ == null) {
+          regionA_ = null;
           onChanged();
         } else {
-          regionBuilder_.addMessage(index, value);
+          regionABuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder addRegion(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.add(builderForValue.build());
-          onChanged();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionABuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegionAFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionAOrBuilder() {
+        if (regionABuilder_ != null) {
+          return regionABuilder_.getMessageOrBuilder();
         } else {
-          regionBuilder_.addMessage(builderForValue.build());
+          return regionA_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionA_;
         }
-        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
        */
-      public Builder addRegion(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.add(index, builderForValue.build());
-          onChanged();
-        } else {
-          regionBuilder_.addMessage(index, builderForValue.build());
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
+          getRegionAFieldBuilder() {
+        if (regionABuilder_ == null) {
+          regionABuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
+                  getRegionA(),
+                  getParentForChildren(),
+                  isClean());
+          regionA_ = null;
         }
-        return this;
+        return regionABuilder_;
       }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionB_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBBuilder_;
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public Builder addAllRegion(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> values) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
-              values, region_);
-          onChanged();
+      public boolean hasRegionB() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionB() {
+        if (regionBBuilder_ == null) {
+          return regionB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionB_;
         } else {
-          regionBuilder_.addAllMessages(values);
+          return regionBBuilder_.getMessage();
         }
-        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public Builder clearRegion() {
-        if (regionBuilder_ == null) {
-          region_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000001);
+      public Builder setRegionB(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionBBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          regionB_ = value;
           onChanged();
         } else {
-          regionBuilder_.clear();
+          regionBBuilder_.setMessage(value);
         }
+        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public Builder removeRegion(int index) {
-        if (regionBuilder_ == null) {
-          ensureRegionIsMutable();
-          region_.remove(index);
+      public Builder setRegionB(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+        if (regionBBuilder_ == null) {
+          regionB_ = builderForValue.build();
           onChanged();
         } else {
-          regionBuilder_.remove(index);
+          regionBBuilder_.setMessage(builderForValue.build());
         }
+        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder(
-          int index) {
-        return getRegionFieldBuilder().getBuilder(index);
-      }
-      /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
-          int index) {
-        if (regionBuilder_ == null) {
-          return region_.get(index);  } else {
-          return regionBuilder_.getMessageOrBuilder(index);
+      public Builder mergeRegionB(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionBBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              regionB_ != null &&
+              regionB_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) {
+            regionB_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(regionB_).mergeFrom(value).buildPartial();
+          } else {
+            regionB_ = value;
+          }
+          onChanged();
+        } else {
+          regionBBuilder_.mergeFrom(value);
         }
+        bitField0_ |= 0x00000002;
+        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-           getRegionOrBuilderList() {
-        if (regionBuilder_ != null) {
-          return regionBuilder_.getMessageOrBuilderList();
+      public Builder clearRegionB() {
+        if (regionBBuilder_ == null) {
+          regionB_ = null;
+          onChanged();
         } else {
-          return java.util.Collections.unmodifiableList(region_);
+          regionBBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder() {
-        return getRegionFieldBuilder().addBuilder(
-            org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getRegionBFieldBuilder().getBuilder();
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder addRegionBuilder(
-          int index) {
-        return getRegionFieldBuilder().addBuilder(
-            index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance());
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionBOrBuilder() {
+        if (regionBBuilder_ != null) {
+          return regionBBuilder_.getMessageOrBuilder();
+        } else {
+          return regionB_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionB_;
+        }
       }
       /**
-       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder> 
-           getRegionBuilderList() {
-        return getRegionFieldBuilder().getBuilderList();
-      }
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
           org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
-          getRegionFieldBuilder() {
-        if (regionBuilder_ == null) {
-          regionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          getRegionBFieldBuilder() {
+        if (regionBBuilder_ == null) {
+          regionBBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
               org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>(
-                  region_,
-                  ((bitField0_ & 0x00000001) == 0x00000001),
+                  getRegionB(),
                   getParentForChildren(),
                   isClean());
-          region_ = null;
+          regionB_ = null;
         }
-        return regionBuilder_;
+        return regionBBuilder_;
       }
 
       private boolean forcible_ ;
@@ -6399,7 +6447,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public boolean hasForcible() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
        * <code>optional bool forcible = 3 [default = false];</code>
@@ -6411,7 +6459,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public Builder setForcible(boolean value) {
-        bitField0_ |= 0x00000002;
+        bitField0_ |= 0x00000004;
         forcible_ = value;
         onChanged();
         return this;
@@ -6420,7 +6468,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public Builder clearForcible() {
-        bitField0_ = (bitField0_ & ~0x00000002);
+        bitField0_ = (bitField0_ & ~0x00000004);
         forcible_ = false;
         onChanged();
         return this;
@@ -6431,7 +6479,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public boolean hasNonceGroup() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
@@ -6443,7 +6491,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public Builder setNonceGroup(long value) {
-        bitField0_ |= 0x00000004;
+        bitField0_ |= 0x00000008;
         nonceGroup_ = value;
         onChanged();
         return this;
@@ -6452,7 +6500,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public Builder clearNonceGroup() {
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000008);
         nonceGroup_ = 0L;
         onChanged();
         return this;
@@ -6463,7 +6511,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public boolean hasNonce() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+        return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
        * <code>optional uint64 nonce = 5 [default = 0];</code>
@@ -6475,7 +6523,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public Builder setNonce(long value) {
-        bitField0_ |= 0x00000008;
+        bitField0_ |= 0x00000010;
         nonce_ = value;
         onChanged();
         return this;
@@ -6484,7 +6532,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public Builder clearNonce() {
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000010);
         nonce_ = 0L;
         onChanged();
         return this;
@@ -6500,46 +6548,46 @@ public final class MasterProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.MergeTableRegionsRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsRequest)
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.MergeTableRegionsRequest)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest DEFAULT_INSTANCE;
+    // @@protoc_insertion_point(class_scope:hbase.pb.DispatchMergingRegionsRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest();
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest();
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstance() {
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<MergeTableRegionsRequest>() {
-      public MergeTableRegionsRequest parsePartialFrom(
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsRequest>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<DispatchMergingRegionsRequest>() {
+      public DispatchMergingRegionsRequest parsePartialFrom(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new MergeTableRegionsRequest(input, extensionRegistry);
+          return new DispatchMergingRegionsRequest(input, extensionRegistry);
       }
     };
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsRequest> parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsRequest> getParserForType() {
       return PARSER;
     }
 
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstanceForType() {
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface MergeTableRegionsResponseOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsResponse)
+  public interface DispatchMergingRegionsResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsResponse)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
@@ -6552,17 +6600,17 @@ public final class MasterProtos {
     long getProcId();
   }
   /**
-   * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
+   * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
    */
-  public  static final class MergeTableRegionsResponse extends
+  public  static final class DispatchMergingRegionsResponse extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsResponse)
-      MergeTableRegionsResponseOrBuilder {
-    // Use MergeTableRegionsResponse.newBuilder() to construct.
-    private MergeTableRegionsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      // @@protoc_insertion_point(message_implements:hbase.pb.DispatchMergingRegionsResponse)
+      DispatchMergingRegionsResponseOrBuilder {
+    // Use DispatchMergingRegionsResponse.newBuilder() to construct.
+    private DispatchMergingRegionsResponse(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private MergeTableRegionsResponse() {
+    private DispatchMergingRegionsResponse() {
       procId_ = 0L;
     }
 
@@ -6571,7 +6619,7 @@ public final class MasterProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private MergeTableRegionsResponse(
+    private DispatchMergingRegionsResponse(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
@@ -6613,14 +6661,14 @@ public final class MasterProtos {
     }
     public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
     }
 
     protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.Builder.class);
     }
 
     private int bitField0_;
@@ -6677,10 +6725,10 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) obj;
 
       boolean result = true;
       result = result && (hasProcId() == other.hasProcId());
@@ -6709,58 +6757,58 @@ public final class MasterProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(byte[] data)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         byte[] data,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseDelimitedFrom(
         java.io.InputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
           .parseWithIOException(PARSER, input);
     }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
         org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
         org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -6772,7 +6820,7 @@ public final class MasterProtos {
     public static Builder newBuilder() {
       return DEFAULT_INSTANCE.toBuilder();
     }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -6787,25 +6835,25 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
+     * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
      */
     public static final class Builder extends
         org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsResponse)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponseOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.DispatchMergingRegionsResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponseOrBuilder {
       public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
       }
 
       protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -6829,23 +6877,23 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse(this);
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -6884,19 +6932,2515 @@ public final class MasterProtos {
         return (Builder) super.addRepeatedField(field, value);
       }
       public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsResponse)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.DispatchMergingRegionsResponse)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsResponse>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<DispatchMergingRegionsResponse>() {
+      public DispatchMergingRegionsResponse parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new DispatchMergingRegionsResponse(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsResponse> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsResponse> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface MergeTableRegionsRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsRequest)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> 
+        getRegionList();
+    /**
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
+    /**
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     */
+    int getRegionCount();
+    /**
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
+        getRegionOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+        int index);
+
+    /**
+     * <code>optional bool forcible = 3 [default = false];</code>
+     */
+    boolean hasForcible();
+    /**
+     * <code>optional bool forcible = 3 [default = false];</code>
+     */
+    boolean getForcible();
+
+    /**
+     * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+     */
+    boolean hasNonceGroup();
+    /**
+     * <code>optional uint64 nonce_group = 4 [default = 0];</code>
+     */
+    long getNonceGroup();
+
+    /**
+     * <code>optional uint64 nonce = 5 [default = 0];</code>
+     */
+    boolean hasNonce();
+    /**
+     * <code>optional uint64 nonce = 5 [default = 0];</code>
+     */
+    long getNonce();
+  }
+  /**
+   * <pre>
+   **
+   * Merging the specified regions in a table.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
+   */
+  public  static final class MergeTableRegionsRequest extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.MergeTableRegionsRequest)
+      MergeTableRegionsRequestOrBuilder {
+    // Use MergeTableRegionsRequest.newBuilder() to construct.
+    private MergeTableRegionsRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private MergeTableRegionsRequest() {
+      region_ = java.util.Collections.emptyList();
+      forcible_ = false;
+      nonceGroup_ = 0L;
+      nonce_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private MergeTableRegionsRequest(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              region_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000001;
+              forcible_ = input.readBool();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000002;
+              nonceGroup_ = input.readUInt64();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000004;
+              nonce_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          region_ = java.util.Collections.unmodifiableList(region_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
+    }
+
+    private int bitField

<TRUNCATED>

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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index e22695b..eeaae97 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -79,6 +79,21 @@ message MoveRegionRequest {
 message MoveRegionResponse {
 }
 
+ /**
+ * Dispatch merging the specified regions.
+ */
+message DispatchMergingRegionsRequest {
+  required RegionSpecifier region_a = 1;
+  required RegionSpecifier region_b = 2;
+  optional bool forcible = 3 [default = false];
+  optional uint64 nonce_group = 4 [default = 0];
+  optional uint64 nonce = 5 [default = 0];
+}
+
+message DispatchMergingRegionsResponse {
+  optional uint64 proc_id = 1;
+}
+
 /**
  * Merging the specified regions in a table.
  */
@@ -117,6 +132,17 @@ message OfflineRegionResponse {
 
 /* Table-level protobufs */
 
+message SplitTableRegionRequest {
+  required RegionInfo region_info = 1;
+  required bytes split_row = 2;
+  optional uint64 nonce_group = 3 [default = 0];
+  optional uint64 nonce = 4 [default = 0];
+}
+
+message SplitTableRegionResponse {
+  optional uint64 proc_id = 1;
+}
+
 message CreateTableRequest {
   required TableSchema table_schema = 1;
   repeated bytes split_keys = 2;
@@ -628,6 +654,10 @@ service MasterService {
   rpc ModifyColumn(ModifyColumnRequest)
     returns(ModifyColumnResponse);
 
+ /** Master dispatch merging the regions */
+  rpc DispatchMergingRegions(DispatchMergingRegionsRequest)
+    returns(DispatchMergingRegionsResponse);
+
   /** Move the region region to the destination server. */
   rpc MoveRegion(MoveRegionRequest)
     returns(MoveRegionResponse);
@@ -658,6 +688,12 @@ service MasterService {
   rpc OfflineRegion(OfflineRegionRequest)
     returns(OfflineRegionResponse);
 
+  /**
+   * Split region
+   */
+  rpc SplitRegion(SplitTableRegionRequest)
+    returns(SplitTableRegionResponse);
+
   /** Deletes a table */
   rpc DeleteTable(DeleteTableRequest)
     returns(DeleteTableResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index ef3f973..6b7206f 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -262,38 +262,31 @@ message RestoreSnapshotStateData {
   repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
 }
 
-enum MergeTableRegionsState {
-  MERGE_TABLE_REGIONS_PREPARE = 1;
-  MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;
-  MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;
-  MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;
-  MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;
-  MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;
-  MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;
-  MERGE_TABLE_REGIONS_UPDATE_META = 8;
-  MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;
-  MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;
-  MERGE_TABLE_REGIONS_POST_OPERATION = 11;
+enum DispatchMergingRegionsState {
+  DISPATCH_MERGING_REGIONS_PREPARE = 1;
+  DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;
+  DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
+  DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;
+  DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;
 }
 
-message MergeTableRegionsStateData {
+message DispatchMergingRegionsStateData {
   required UserInformation user_info = 1;
-  repeated RegionInfo region_info = 2;
-  required RegionInfo merged_region_info = 3;
-  optional bool forcible = 4 [default = false];
+  required TableName table_name = 2;
+  repeated RegionInfo region_info = 3;
+  optional bool forcible = 4;
 }
 
 enum SplitTableRegionState {
   SPLIT_TABLE_REGION_PREPARE = 1;
   SPLIT_TABLE_REGION_PRE_OPERATION = 2;
-  SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;
-  SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;
-  SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;
-  SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;
-  SPLIT_TABLE_REGION_UPDATE_META = 7;
-  SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;
-  SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;
-  SPLIT_TABLE_REGION_POST_OPERATION = 10;
+  SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 3;
+  SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 4;
+  SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 5;
+  SPLIT_TABLE_REGION_UPDATE_META = 6;
+  SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 7;
+  SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 8;
+  SPLIT_TABLE_REGION_POST_OPERATION = 9;
 }
 
 message SplitTableRegionStateData {
@@ -302,6 +295,29 @@ message SplitTableRegionStateData {
   repeated RegionInfo child_region_info = 3;
 }
 
+enum MergeTableRegionsState {
+  MERGE_TABLE_REGIONS_PREPARE = 1;
+  MERGE_TABLE_REGIONS_PRE_OPERATION = 2;
+  MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 3;
+  MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 4;
+  MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 5;
+  MERGE_TABLE_REGIONS_CLOSE_REGIONS = 6;
+  MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 7;
+  MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 8;
+  MERGE_TABLE_REGIONS_UPDATE_META = 9;
+  MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 10;
+  MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 11;
+  MERGE_TABLE_REGIONS_POST_OPERATION = 12;
+}
+
+message MergeTableRegionsStateData {
+  required UserInformation user_info = 1;
+  repeated RegionInfo region_info = 2;
+  optional RegionInfo merged_region_info = 3;
+  optional bool forcible = 4 [default = false];
+}
+
+
 message ServerCrashStateData {
   required ServerName server_name = 1;
   optional bool distributed_log_replay = 2;
@@ -323,3 +339,34 @@ enum ServerCrashState {
   SERVER_CRASH_WAIT_ON_ASSIGN = 9;
   SERVER_CRASH_FINISH = 100;
 }
+
+enum RegionTransitionState {
+  REGION_TRANSITION_QUEUE = 1;
+  REGION_TRANSITION_DISPATCH = 2;
+  REGION_TRANSITION_FINISH = 3;
+}
+
+message AssignRegionStateData {
+  required RegionTransitionState transition_state = 1;
+  required RegionInfo region_info = 2;
+  optional bool force_new_plan = 3 [default = false];
+  optional ServerName target_server = 4;
+}
+
+message UnassignRegionStateData {
+  required RegionTransitionState transition_state = 1;
+  required RegionInfo region_info = 2;
+  optional ServerName destination_server = 3;
+  optional bool force = 4 [default = false];
+}
+
+enum MoveRegionState {
+  MOVE_REGION_UNASSIGN = 1;
+  MOVE_REGION_ASSIGN = 2;
+}
+
+message MoveRegionStateData {
+  required RegionInfo region_info = 1;
+  required ServerName source_server = 2;
+  required ServerName destination_server = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index 1c373ee..60cf77a 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -26,7 +26,6 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "HBase.proto";
-import "Master.proto";
 import "ClusterStatus.proto";
 
 message RegionServerStartupRequest {
@@ -127,20 +126,6 @@ message ReportRegionStateTransitionResponse {
   optional string error_message = 1;
 }
 
-/**
- * Splits the specified region.
- */
-message SplitTableRegionRequest {
-  required RegionInfo region_info = 1;
-  required bytes split_row = 2;
-  optional uint64 nonce_group = 3 [default = 0];
-  optional uint64 nonce = 4 [default = 0];
-}
-
-message SplitTableRegionResponse {
-  optional uint64 proc_id = 1;
-}
-
 service RegionServerStatusService {
   /** Called when a region server first starts. */
   rpc RegionServerStartup(RegionServerStartupRequest)
@@ -170,16 +155,4 @@ service RegionServerStatusService {
    */
   rpc ReportRegionStateTransition(ReportRegionStateTransitionRequest)
     returns(ReportRegionStateTransitionResponse);
-
-  /**
-   * Split region
-   */
-  rpc SplitRegion(SplitTableRegionRequest)
-    returns(SplitTableRegionResponse);
-
-  /**
-   * Get procedure result
-   */
-  rpc getProcedureResult(GetProcedureResultRequest)
-    returns(GetProcedureResultResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
index 3c0cccf..05ed333 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminServer.java
@@ -37,7 +37,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
@@ -121,10 +122,9 @@ public class RSGroupAdminServer implements RSGroupAdmin {
         addRegion(regions, el.getKey());
       }
     }
-    for (RegionState state:
-        this.master.getAssignmentManager().getRegionStates().getRegionsInTransition()) {
-      if (state.getServerName().getAddress().equals(server)) {
-        addRegion(regions, state.getRegion());
+    for (RegionStateNode state : master.getAssignmentManager().getRegionsInTransition()) {
+      if (state.getRegionLocation().getAddress().equals(server)) {
+        addRegion(regions, state.getRegionInfo());
       }
     }
     return regions;
@@ -531,7 +531,7 @@ public class RSGroupAdminServer implements RSGroupAdmin {
         LOG.info("RSGroup balance " + groupName + " starting with plan count: " + plans.size());
         for (RegionPlan plan: plans) {
           LOG.info("balance " + plan);
-          assignmentManager.balance(plan);
+          assignmentManager.moveAsync(plan);
         }
         LOG.info("RSGroup balance " + groupName + " completed after " +
             (System.currentTimeMillis()-startTime) + " seconds");

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 30efc0a..5cdfad2 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -138,7 +138,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
             }
           }
         }
-        groupClusterLoad.put(TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME), groupClusterState);
+        groupClusterLoad.put(HConstants.ENSEMBLE_TABLE_NAME, groupClusterState);
         this.internalBalancer.setClusterLoad(groupClusterLoad);
         List<RegionPlan> groupPlans = this.internalBalancer
             .balanceCluster(groupClusterState);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
index 83fe122..0f1e849 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRSGroupBasedLoadBalancer.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
-import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
index 76a85a9..b5e6dd0 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/AssignmentManagerStatusTmpl.jamon
@@ -18,7 +18,9 @@ limitations under the License.
 </%doc>
 <%import>
 org.apache.hadoop.hbase.HRegionInfo;
-org.apache.hadoop.hbase.master.AssignmentManager;
+org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+org.apache.hadoop.hbase.master.assignment.AssignmentManager.RegionInTransitionStat;
+org.apache.hadoop.hbase.master.assignment.RegionStates.RegionFailedOpen;
 org.apache.hadoop.hbase.master.RegionState;
 org.apache.hadoop.conf.Configuration;
 org.apache.hadoop.hbase.HBaseConfiguration;
@@ -35,28 +37,12 @@ int limit = 100;
 
 <%java SortedSet<RegionState> rit = assignmentManager
   .getRegionStates().getRegionsInTransitionOrderedByTimestamp();
-  Map<String, AtomicInteger> failedRegionTracker = assignmentManager.getFailedOpenTracker();
-   %>
+%>
 
 <%if !rit.isEmpty() %>
 <%java>
-HashSet<String> ritsOverThreshold = new HashSet<String>();
-HashSet<String> ritsTwiceThreshold = new HashSet<String>();
-// process the map to find region in transition details
-Configuration conf = HBaseConfiguration.create();
-int ritThreshold = conf.getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
-int numOfRITOverThreshold = 0;
 long currentTime = System.currentTimeMillis();
-for (RegionState rs : rit) {
-  long ritTime = currentTime - rs.getStamp();
-  if(ritTime > (ritThreshold * 2)) {
-     numOfRITOverThreshold++;
-     ritsTwiceThreshold.add(rs.getRegion().getEncodedName());
-  } else if (ritTime > ritThreshold) {
-     numOfRITOverThreshold++;
-     ritsOverThreshold.add(rs.getRegion().getEncodedName());
-  }
-}
+RegionInTransitionStat ritStat = assignmentManager.computeRegionInTransitionStat();
 
 int numOfRITs = rit.size();
 int ritsPerPage = Math.min(5, numOfRITs);
@@ -65,15 +51,15 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
     <section>
     <h2>Regions in Transition</h2>
      <p><% numOfRITs %> region(s) in transition.
-     <%if !ritsTwiceThreshold.isEmpty()  %>
+     <%if ritStat.hasRegionsTwiceOverThreshold()  %>
          <span class="label label-danger" style="font-size:100%;font-weight:normal">
-     <%elseif !ritsOverThreshold.isEmpty() %>
+     <%elseif ritStat.hasRegionsOverThreshold() %>
          <span class="label label-warning" style="font-size:100%;font-weight:normal">
      <%else>
          <span>
      </%if>
-         <% numOfRITOverThreshold %> region(s) in transition for
-             more than <% ritThreshold %> milliseconds.
+         <% ritStat.getTotalRITsOverThreshold() %> region(s) in transition for
+             more than <% ritStat.getRITThreshold() %> milliseconds.
          </span>
      </p>
      <div class="tabbable">
@@ -90,25 +76,26 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
                      <th>State</th><th>RIT time (ms)</th> <th>Retries </th></tr>
              </%if>
 
-             <%if ritsOverThreshold.contains(rs.getRegion().getEncodedName()) %>
-                     <tr class="alert alert-warning" role="alert">
-             <%elseif ritsTwiceThreshold.contains(rs.getRegion().getEncodedName()) %>
+             <%if ritStat.isRegionTwiceOverThreshold(rs.getRegion()) %>
                      <tr class="alert alert-danger" role="alert">
+             <%elseif ritStat.isRegionOverThreshold(rs.getRegion()) %>
+                     <tr class="alert alert-warning" role="alert">
             <%else>
                     <tr>
             </%if>
                         <%java>
                           String retryStatus = "0";
-                          AtomicInteger numOpenRetries = failedRegionTracker.get(
-                              rs.getRegion().getEncodedName());
-                          if (numOpenRetries != null ) {
-                            retryStatus = Integer.toString(numOpenRetries.get());
+                          RegionFailedOpen regionFailedOpen = assignmentManager
+                            .getRegionStates().getFailedOpen(rs.getRegion());
+                          if (regionFailedOpen != null) {
+                            retryStatus = Integer.toString(regionFailedOpen.getRetries());
                           } else if (rs.getState() ==  RegionState.State.FAILED_OPEN) {
-                             retryStatus = "Failed";
+                            retryStatus = "Failed";
                           }
                         </%java>
                         <td><% rs.getRegion().getEncodedName() %></td><td>
-                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs, conf) %></td>
+                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs,
+                            assignmentManager.getConfiguration()) %></td>
                         <td><% (currentTime - rs.getStamp()) %> </td>
                         <td> <% retryStatus %> </td>
                      </tr>

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 36d5112..0e76455 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -41,7 +41,7 @@ org.apache.hadoop.hbase.TableName;
 org.apache.hadoop.hbase.client.Admin;
 org.apache.hadoop.hbase.client.MasterSwitchType;
 org.apache.hadoop.hbase.client.SnapshotDescription;
-org.apache.hadoop.hbase.master.AssignmentManager;
+org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 org.apache.hadoop.hbase.master.DeadServer;
 org.apache.hadoop.hbase.master.HMaster;
 org.apache.hadoop.hbase.master.RegionState;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
index ed1ae31..4f134c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/VersionInfoUtil.java
@@ -35,9 +35,7 @@ public final class VersionInfoUtil {
   }
 
   public static boolean currentClientHasMinimumVersion(int major, int minor) {
-    RpcCallContext call = RpcServer.getCurrentCall();
-    HBaseProtos.VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
-    return hasMinimumVersion(versionInfo, major, minor);
+    return hasMinimumVersion(getCurrentClientVersionInfo(), major, minor);
   }
 
   public static boolean hasMinimumVersion(HBaseProtos.VersionInfo versionInfo,
@@ -53,7 +51,7 @@ public final class VersionInfoUtil {
         return clientMinor >= minor;
       }
       try {
-        String[] components = versionInfo.getVersion().split("\\.");
+        final String[] components = getVersionComponents(versionInfo);
 
         int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
         if (clientMajor != major) {
@@ -68,4 +66,79 @@ public final class VersionInfoUtil {
     }
     return false;
   }
+
+  /**
+   * @return the versionInfo extracted from the current RpcCallContext
+   */
+  private static HBaseProtos.VersionInfo getCurrentClientVersionInfo() {
+    RpcCallContext call = RpcServer.getCurrentCall();
+    return call != null ? call.getClientVersionInfo() : null;
+  }
+
+  /**
+   * @return the version number extracted from the current RpcCallContext as int.
+   *         (e.g. 0x0103004 is 1.3.4)
+   */
+  public static int getCurrentClientVersionNumber() {
+    return getVersionNumber(getCurrentClientVersionInfo());
+  }
+
+
+  /**
+   * @param version
+   * @return the passed-in <code>version</code> int as a version String
+   *         (e.g. 0x0103004 is 1.3.4)
+   */
+  public static String versionNumberToString(final int version) {
+    return String.format("%d.%d.%d",
+        ((version >> 20) & 0xff),
+        ((version >> 12) & 0xff),
+        (version & 0xfff));
+  }
+
+  /**
+   * Pack the full number version in a int. by shifting each component by 8bit,
+   * except the dot release which has 12bit.
+   * Examples: (1.3.4 is 0x0103004, 2.1.0 is 0x0201000)
+   * @param versionInfo the VersionInfo object to pack
+   * @return the version number as int. (e.g. 0x0103004 is 1.3.4)
+   */
+  private static int getVersionNumber(final HBaseProtos.VersionInfo versionInfo) {
+    if (versionInfo != null) {
+      try {
+        final String[] components = getVersionComponents(versionInfo);
+        int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
+        int clientMinor = components.length > 1 ? Integer.parseInt(components[1]) : 0;
+        int clientPatch = components.length > 2 ? Integer.parseInt(components[2]) : 0;
+        return buildVersionNumber(clientMajor, clientMinor, clientPatch);
+      } catch (NumberFormatException e) {
+        int clientMajor = versionInfo.hasVersionMajor() ? versionInfo.getVersionMajor() : 0;
+        int clientMinor = versionInfo.hasVersionMinor() ? versionInfo.getVersionMinor() : 0;
+        return buildVersionNumber(clientMajor, clientMinor, 0);
+      }
+    }
+    return(0); // no version
+  }
+
+  /**
+   * Pack the full number version in a int. by shifting each component by 8bit,
+   * except the dot release which has 12bit.
+   * Examples: (1.3.4 is 0x0103004, 2.1.0 is 0x0201000)
+   * @param major version major number
+   * @param minor version minor number
+   * @param patch version patch number
+   * @return the version number as int. (e.g. 0x0103004 is 1.3.4)
+   */
+  private static int buildVersionNumber(int major, int minor, int patch) {
+    return (major << 20) | (minor << 12) | patch;
+  }
+
+  /**
+   * Returns the version components
+   * Examples: "1.2.3" returns [1, 2, 3], "4.5.6-SNAPSHOT" returns [4, 5, 6, "SNAPSHOT"]
+   * @returns the components of the version string
+   */
+  private static String[] getVersionComponents(final HBaseProtos.VersionInfo versionInfo) {
+    return versionInfo.getVersion().split("[\\.-]");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
index f792b36..558c9c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BalancedQueueRpcExecutor.java
@@ -17,11 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import java.util.List;
 import java.util.concurrent.BlockingQueue;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
index ab16627..b3df28a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
@@ -120,4 +120,4 @@ public class FastPathBalancedQueueRpcExecutor extends BalancedQueueRpcExecutor {
       return true;
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index d51d83b..2ee2d7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -37,8 +35,6 @@ import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 @InterfaceStability.Evolving
 public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObserver {
-  private static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
-
   private int port;
   private final PriorityFunction priority;
   private final RpcExecutor callExecutor;
@@ -82,14 +78,14 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
 
     if (callqReadShare > 0) {
       // at least 1 read handler and 1 write handler
-      callExecutor = new RWQueueRpcExecutor("deafult.RWQ", Math.max(2, handlerCount),
+      callExecutor = new RWQueueRpcExecutor("default.RWQ", Math.max(2, handlerCount),
         maxQueueLength, priority, conf, server);
     } else {
       if (RpcExecutor.isFifoQueueType(callQueueType) || RpcExecutor.isCodelQueueType(callQueueType)) {
-        callExecutor = new FastPathBalancedQueueRpcExecutor("deafult.FPBQ", handlerCount,
+        callExecutor = new FastPathBalancedQueueRpcExecutor("default.FPBQ", handlerCount,
             maxQueueLength, priority, conf, server);
       } else {
-        callExecutor = new BalancedQueueRpcExecutor("deafult.BQ", handlerCount, maxQueueLength,
+        callExecutor = new BalancedQueueRpcExecutor("default.BQ", handlerCount, maxQueueLength,
             priority, conf, server);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
index 5f90d50..c409f6e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
@@ -1980,8 +1980,8 @@ public class SimpleRpcServer extends RpcServer {
           if (!running) {
             return;
           }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(Thread.currentThread().getName()+": task running");
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("running");
           }
           try {
             closeIdle(false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8faab93a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
deleted file mode 100644
index 4513a5d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
+++ /dev/null
@@ -1,49 +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;
-
-import java.util.concurrent.Callable;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HRegionInfo;
-
-/**
- * A callable object that invokes the corresponding action that needs to be
- * taken for assignment of a region in transition. 
- * Implementing as future callable we are able to act on the timeout
- * asynchronously.
- */
-@InterfaceAudience.Private
-public class AssignCallable implements Callable<Object> {
-  private AssignmentManager assignmentManager;
-
-  private HRegionInfo hri;
-
-  public AssignCallable(
-      AssignmentManager assignmentManager, HRegionInfo hri) {
-    this.assignmentManager = assignmentManager;
-    this.hri = hri;
-  }
-
-  @Override
-  public Object call() throws Exception {
-    assignmentManager.assign(hri);
-    return null;
-  }
-}