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/05/25 06:32:05 UTC

[01/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Repository: hbase
Updated Branches:
  refs/heads/master dc1065a85 -> a3c5a7448


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 2630ce0..81846df 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.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -66,27 +66,23 @@ 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.MasterCoprocessorEnvironment;
 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.exceptions.UnexpectedStateException;
+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.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+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.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;
@@ -102,11 +98,11 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 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
@@ -114,9 +110,8 @@ import org.junit.rules.TestRule;
 @Category({RegionServerTests.class, LargeTests.class})
 @SuppressWarnings("deprecation")
 public class TestSplitTransactionOnCluster {
-  private static final Log LOG = LogFactory.getLog(TestSplitTransactionOnCluster.class);
-  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
-      withLookingForStuckThread(true).build();
+  private static final Log LOG =
+    LogFactory.getLog(TestSplitTransactionOnCluster.class);
   private Admin admin = null;
   private MiniHBaseCluster cluster = null;
   private static final int NB_SERVERS = 3;
@@ -155,11 +150,8 @@ public class TestSplitTransactionOnCluster {
       throws IOException, InterruptedException {
     assertEquals(1, regions.size());
     HRegionInfo hri = regions.get(0).getRegionInfo();
-    try {
-      cluster.getMaster().getAssignmentManager().waitForAssignment(hri, 600000);
-    } catch (NoSuchProcedureException e) {
-      LOG.info("Presume the procedure has been cleaned up so just proceed: " + e.toString());
-    }
+    cluster.getMaster().getAssignmentManager()
+      .waitOnRegionToClearRegionsInTransition(hri, 600000);
     return hri;
   }
 
@@ -168,12 +160,24 @@ public class TestSplitTransactionOnCluster {
       final Region region,
       final byte[] midKey) throws IOException {
     long procId = cluster.getMaster().splitRegion(region.getRegionInfo(), midKey, 0, 0);
-    // 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);
+    // 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.");
+    }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testRITStateForRollback() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final HMaster master = cluster.getMaster();
@@ -211,10 +215,7 @@ public class TestSplitTransactionOnCluster {
       observer.latch.await();
 
       LOG.info("Waiting for region to come out of RIT");
-      while (!cluster.getMaster().getAssignmentManager().getRegionStates().isRegionOnline(hri)) {
-        Threads.sleep(100);
-      }
-      assertTrue(cluster.getMaster().getAssignmentManager().getRegionStates().isRegionOnline(hri));
+      cluster.getMaster().getAssignmentManager().waitOnRegionToClearRegionsInTransition(hri, 60000);
     } finally {
       admin.setBalancerRunning(true, false);
       master.setCatalogJanitorEnabled(true);
@@ -223,7 +224,7 @@ public class TestSplitTransactionOnCluster {
     }
   }
 
-  @Test
+  @Test(timeout = 60000)
   public void testSplitFailedCompactionAndSplit() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     // Create table then get the single region for our new table.
@@ -283,8 +284,8 @@ public class TestSplitTransactionOnCluster {
     }
   }
 
-  @Test
-  public void testSplitRollbackOnRegionClosing() throws IOException, InterruptedException {
+  @Test (timeout = 300000)
+  public void testExistingZnodeBlocksSplitAndWeRollback() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
     // Create table then get the single region for our new table.
@@ -320,7 +321,8 @@ public class TestSplitTransactionOnCluster {
         assertEquals(regionCount, ProtobufUtil.getOnlineRegions(
           server.getRSRpcServices()).size());
       }
-      regionStates.updateRegionState(hri, State.OPEN);
+      regionStates.regionOnline(hri, server.getServerName());
+
       // Now try splitting and it should work.
       split(hri, server, regionCount);
       // Get daughters
@@ -339,7 +341,8 @@ public class TestSplitTransactionOnCluster {
    * @throws IOException
    * @throws InterruptedException
    */
-  @Test
+  @Ignore // TODO: revisit this test when the new AM and SSH is implement
+  @Test (timeout=300000)
   public void testShutdownFixupWhenDaughterHasSplit()throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
@@ -416,7 +419,7 @@ public class TestSplitTransactionOnCluster {
     }
   }
 
-  @Test
+  @Test(timeout = 180000)
   public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
     TableName userTableName = TableName.valueOf(name.getMethodName());
     HTableDescriptor htd = new HTableDescriptor(userTableName);
@@ -441,7 +444,6 @@ 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());
@@ -459,13 +461,10 @@ public class TestSplitTransactionOnCluster {
           .getRegionsOfTable(userTableName);
 
       while (regionsOfTable.size() != 2) {
-        Thread.sleep(1000);
+        Thread.sleep(2000);
         regionsOfTable = cluster.getMaster()
             .getAssignmentManager().getRegionStates()
             .getRegionsOfTable(userTableName);
-        LOG.debug("waiting 2 regions to be available, got " + regionsOfTable.size() +
-          ": " + regionsOfTable);
-
       }
       Assert.assertEquals(2, regionsOfTable.size());
 
@@ -489,7 +488,7 @@ public class TestSplitTransactionOnCluster {
    * @throws NodeExistsException
    * @throws KeeperException
    */
-  @Test
+  @Test (timeout = 300000)
   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
       throws IOException, InterruptedException, NodeExistsException,
       KeeperException, ServiceException {
@@ -512,35 +511,25 @@ public class TestSplitTransactionOnCluster {
       // Get region pre-split.
       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
       printOutRegions(server, "Initial regions: ");
-      // Call split.
+
       this.admin.splitRegion(hri.getRegionName());
-      List<HRegion> daughters = checkAndGetDaughters(tableName);
-      // Before cleanup, get a new master.
+      checkAndGetDaughters(tableName);
+
       HMaster master = abortAndWaitForMaster();
-      // Now call compact on the daughters and clean up any references.
-      for (HRegion daughter: daughters) {
-        daughter.compact(true);
-        assertFalse(daughter.hasReferences());
-      }
-      // BUT calling compact on the daughters is not enough. The CatalogJanitor looks
-      // in the filesystem, and the filesystem content is not same as what the Region
-      // is reading from. Compacted-away files are picked up later by the compacted
-      // file discharger process. It runs infrequently. Make it run so CatalogJanitor
-      // doens't find any references.
-      for (RegionServerThread rst: cluster.getRegionServerThreads()) {
-        boolean oldSetting = rst.getRegionServer().compactedFileDischarger.setUseExecutor(false);
-        rst.getRegionServer().compactedFileDischarger.run();
-        rst.getRegionServer().compactedFileDischarger.setUseExecutor(oldSetting);
-      }
-      cluster.getMaster().setCatalogJanitorEnabled(true);
-      LOG.info("Starting run of CatalogJanitor");
-      cluster.getMaster().getCatalogJanitor().run();
-      ProcedureTestingUtility.waitAllProcedures(cluster.getMaster().getMasterProcedureExecutor());
+
+      this.admin = TESTING_UTIL.getAdmin();
+
+      // Update the region to be offline and split, so that HRegionInfo#equals
+      // returns true in checking rebuilt region states map.
+      hri.setOffline(true);
+      hri.setSplit(true);
       RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+      assertTrue("Split parent should be in SPLIT state",
+        regionStates.isRegionInState(hri, State.SPLIT));
       ServerName regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
-      assertEquals(null, regionServerOfRegion);
+      assertTrue(regionServerOfRegion == null);
     } finally {
-      TESTING_UTIL.getAdmin().setBalancerRunning(true, false);
+      this.admin.setBalancerRunning(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
     }
@@ -640,7 +629,7 @@ public class TestSplitTransactionOnCluster {
    * If a table has regions that have no store files in a region, they should split successfully
    * into two regions with no store files.
    */
-  @Test
+  @Test(timeout = 60000)
   public void testSplitRegionWithNoStoreFiles()
       throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
@@ -679,10 +668,10 @@ public class TestSplitTransactionOnCluster {
         fail("Split execution should have succeeded with no exceptions thrown");
       }
 
-      // Postcondition: split the table with no store files into two regions, but still have no
+      // Postcondition: split the table with no store files into two regions, but still have not
       // store files
       List<HRegion> daughters = cluster.getRegions(tableName);
-      assertEquals(2, daughters.size());
+      assertTrue(daughters.size() == 2);
 
       // check dirs
       HBaseFsck.debugLsr(conf, new Path("/"));
@@ -696,13 +685,10 @@ 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);
@@ -711,12 +697,7 @@ public class TestSplitTransactionOnCluster {
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
 
       // We should not be able to unassign it either
-      try {
-        am.unassign(hri);
-        fail("Should have thrown exception");
-      } catch (UnexpectedStateException e) {
-        // Expected
-      }
+      am.unassign(hri, null);
       assertFalse("Split region can't be unassigned",
         regionStates.isRegionInTransition(hri));
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
@@ -958,14 +939,11 @@ public class TestSplitTransactionOnCluster {
       if (enabled.get() && req.getTransition(0).getTransitionCode().equals(
           TransitionCode.READY_TO_SPLIT) && !resp.hasErrorMessage()) {
         RegionStates regionStates = myMaster.getAssignmentManager().getRegionStates();
-        for (RegionStates.RegionStateNode regionState:
-          regionStates.getRegionsInTransition()) {
-          /* TODO!!!!
+        for (RegionState regionState: regionStates.getRegionsInTransition()) {
           // 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/a3c5a744/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 9ccfeef..5bc4c9b 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,7 +34,6 @@ 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;
@@ -59,17 +58,13 @@ 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/a3c5a744/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 d31d8cb..5b8b404 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,7 +18,6 @@
 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;
@@ -26,14 +25,10 @@ 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/a3c5a744/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 2758d4d..e2aa580 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,22 +22,16 @@ 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.Test;
 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 {
@@ -54,4 +48,4 @@ public class TestWALReplay extends AbstractTestWALReplay {
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);
     return wal;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java
index d8666b6..4bb97d3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController3.java
@@ -57,11 +57,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
 /**
- * Performs checks for reference counting w.r.t. TableAuthManager which is used by
- * AccessController.
- *
- * NOTE: Only one test in  here. In AMv2, there is problem deleting because
- * we are missing auth. For now disabled. See the cleanup method.
+ * Performs checks for reference counting w.r.t. TableAuthManager which is used by AccessController.
  */
 @Category({SecurityTests.class, MediumTests.class})
 public class TestAccessController3 extends SecureTestUtil {
@@ -204,7 +200,7 @@ public class TestAccessController3 extends SecureTestUtil {
       TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
       rs = thread.getRegionServer();
     }
-    // cleanUp();
+    cleanUp();
     TEST_UTIL.shutdownMiniCluster();
     assertTrue("region server should have aborted due to FaultyAccessController", rs.isAborted());
   }
@@ -266,16 +262,12 @@ public class TestAccessController3 extends SecureTestUtil {
 
   private static void cleanUp() throws Exception {
     // Clean the _acl_ table
-    // TODO: Skipping delete because of access issues w/ AMv2.
-    // AMv1 seems to crash servers on exit too for same lack of
-    // auth perms but it gets hung up.
-    /*
     try {
       deleteTable(TEST_UTIL, TEST_TABLE);
     } catch (TableNotFoundException ex) {
       // Test deleted the table, no problem
       LOG.info("Test deleted table " + TEST_TABLE);
-    }*/
+    }
     // Verify all table/namespace permissions are erased
     assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
     assertEquals(

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 f6e328e..02bd49b 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,9 +61,8 @@ 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;
@@ -92,7 +91,6 @@ 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/a3c5a744/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 ca8bc91..b6a185b 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.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/a3c5a744/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 4188146..74ef414 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
@@ -19,30 +19,6 @@
 
 package org.apache.hadoop.hbase.util;
 
-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.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -69,8 +45,11 @@ 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.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
+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.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
@@ -93,7 +72,26 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-@Ignore // Turning off because needs fsck.
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
+import static org.junit.Assert.*;
+
 @Category({MiscTests.class, LargeTests.class})
 public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
   @Rule
@@ -1597,6 +1595,72 @@ 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/a3c5a744/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 3d0647e..403bf5e 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,14 +31,13 @@ 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -59,7 +58,6 @@ import java.util.concurrent.TimeUnit;
 import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
 import static org.junit.Assert.*;
 
-@Ignore
 @Category({MiscTests.class, LargeTests.class})
 public class TestHBaseFsckReplicas extends BaseTestHBaseFsck {
   @Rule

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 7f891d8..6f90bb2 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,13 +19,7 @@
 
 package org.apache.hadoop.hbase.util;
 
-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 com.google.common.collect.Multimap;
 
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.SynchronousQueue;
@@ -41,6 +35,8 @@ 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;
@@ -50,15 +46,14 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.google.common.collect.Multimap;
+import static org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.*;
+import static org.junit.Assert.*;
 
-@Ignore // Until after HBASE-14614 goes in.
 @Category({MiscTests.class, LargeTests.class})
 public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
   @Rule
@@ -83,7 +78,8 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
 
     hbfsckExecutorService = new ScheduledThreadPoolExecutor(POOL_SIZE);
 
-    assignmentManager = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
+    AssignmentManager assignmentManager =
+        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
     regionStates = assignmentManager.getRegionStates();
 
     connection = (ClusterConnection) TEST_UTIL.getConnection();
@@ -112,7 +108,7 @@ public class TestHBaseFsckTwoRS extends BaseTestHBaseFsck {
   public void testFixAssignmentsWhenMETAinTransition() throws Exception {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     admin.closeRegion(cluster.getServerHoldingMeta(), HRegionInfo.FIRST_META_REGIONINFO);
-    assignmentManager.offlineRegion(HRegionInfo.FIRST_META_REGIONINFO);
+    regionStates.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
     new MetaTableLocator().deleteMetaLocation(cluster.getMaster().getZooKeeper());
     assertFalse(regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
     HBaseFsck hbck = doFsck(conf, true);
@@ -397,6 +393,7 @@ 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);
@@ -442,7 +439,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.getAddress().toString()));
+            Bytes.toBytes(serverName.getHostAndPort()));
         meta.put(put);
       }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
index 1560efe..3be7787 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildBase.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 /**
@@ -53,7 +52,7 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
   private static final Log LOG = LogFactory.getLog(TestOfflineMetaRebuildBase.class);
 
   @SuppressWarnings("deprecation")
-  @Ignore @Test(timeout = 120000) // To fix post HBASE-14614
+  @Test(timeout = 120000)
   public void testMetaRebuild() throws Exception {
     wipeOutMeta();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
index 60c4b25..b8565e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildHole.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -39,7 +38,6 @@ import org.junit.experimental.categories.Category;
  * This builds a table, removes info from meta, and then fails when attempting
  * to rebuild meta.
  */
-@Ignore
 @Category({MiscTests.class, MediumTests.class})
 public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore {
   private final static Log LOG = LogFactory.getLog(TestOfflineMetaRebuildHole.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
index 85e0560..ae72935 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/TestOfflineMetaRebuildOverlap.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -42,7 +41,6 @@ import com.google.common.collect.Multimap;
  * This builds a table, builds an overlap, and then fails when attempting to
  * rebuild meta.
  */
-@Ignore
 @Category({MiscTests.class, MediumTests.class})
 public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore {
   private final static Log LOG = LogFactory.getLog(TestOfflineMetaRebuildOverlap.class);


[18/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..69ebd97
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -0,0 +1,3053 @@
+/**
+ *
+ * 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 (tab

<TRUNCATED>

[20/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 299b55e..b886f5c 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,6 +8822,1348 @@ 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) + getDescriptor().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) + getDescriptor().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;
+    }
+
+  }
+
   public interface RegionSpaceUseOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.RegionSpaceUse)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -9482,7 +10824,7 @@ public final class RegionServerStatusProtos {
        * <code>optional .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> 
+          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<
@@ -9598,7 +10940,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse>
         getSpaceUseList();
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
@@ -9611,7 +10953,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
         getSpaceUseOrBuilderList();
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
@@ -9714,7 +11056,7 @@ public final class RegionServerStatusProtos {
     /**
      * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
         getSpaceUseOrBuilderList() {
       return spaceUse_;
     }
@@ -10026,7 +11368,7 @@ public final class RegionServerStatusProtos {
               spaceUseBuilder_ = null;
               spaceUse_ = other.spaceUse_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              spaceUseBuilder_ = 
+              spaceUseBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSpaceUseFieldBuilder() : null;
             } else {
@@ -10262,7 +11604,7 @@ public final class RegionServerStatusProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
            getSpaceUseOrBuilderList() {
         if (spaceUseBuilder_ != null) {
           return spaceUseBuilder_.getMessageOrBuilderList();
@@ -10288,12 +11630,12 @@ public final class RegionServerStatusProtos {
       /**
        * <code>repeated .hbase.pb.RegionSpaceUse space_use = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder>
            getSpaceUseBuilderList() {
         return getSpaceUseFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseOrBuilder>
           getSpaceUseFieldBuilder() {
         if (spaceUseBuilder_ == null) {
           spaceUseBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -10806,6 +12148,32 @@ public final class RegionServerStatusProtos {
       /**
        * <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);
+
+      /**
+       * <pre>
+       **
        * Reports Region filesystem space use
        * </pre>
        *
@@ -10862,6 +12230,22 @@ public final class RegionServerStatusProtos {
         }
 
         @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);
+        }
+
+        @java.lang.Override
         public  void reportRegionSpaceUse(
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
             org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request,
@@ -10902,6 +12286,10 @@ public final class RegionServerStatusProtos {
             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);
+            case 7:
               return impl.reportRegionSpaceUse(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -10928,6 +12316,10 @@ public final class RegionServerStatusProtos {
             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();
+            case 7:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -10954,6 +12346,10 @@ public final class RegionServerStatusProtos {
             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();
+            case 7:
               return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
@@ -11033,6 +12429,32 @@ public final class RegionServerStatusProtos {
     /**
      * <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);
+
+    /**
+     * <pre>
+     **
      * Reports Region filesystem space use
      * </pre>
      *
@@ -11091,6 +12513,16 @@ public final class RegionServerStatusProtos {
               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;
+        case 7:
           this.reportRegionSpaceUse(controller, (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest)request,
             org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse>specializeCallback(
               done));
@@ -11120,6 +12552,10 @@ public final class RegionServerStatusProtos {
         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();
+        case 7:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -11146,6 +12582,10 @@ public final class RegionServerStatusProtos {
         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();
+        case 7:
           return org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
@@ -11243,12 +12683,42 @@ public final class RegionServerStatusProtos {
             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  void reportRegionSpaceUse(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request,
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse> done) {
         channel.callMethod(
-          getDescriptor().getMethods().get(5),
+          getDescriptor().getMethods().get(7),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance(),
@@ -11290,6 +12760,16 @@ public final class RegionServerStatusProtos {
           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;
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse reportRegionSpaceUse(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request)
@@ -11363,12 +12843,36 @@ public final class RegionServerStatusProtos {
       }
 
 
+      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());
+      }
+
+
       public org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse reportRegionSpaceUse(
           org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest request)
           throws org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException {
         return (org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(5),
+          getDescriptor().getMethods().get(7),
           controller,
           request,
           org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse.getDefaultInstance());
@@ -11435,18 +12939,28 @@ public final class RegionServerStatusProtos {
     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_RegionSpaceUse_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;
+  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+    internal_static_hbase_pb_RegionSpaceUse_descriptor;
+  private static final
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpaceUseReportRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUseReportRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RegionSpaceUseReportResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RegionSpaceUseReportResponse_fieldAccessorTable;
 
@@ -11459,61 +12973,70 @@ public final class RegionServerStatusProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\030RegionServerStatus.proto\022\010hbase.pb\032\013HB" +
-      "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(" +
-      "\t\"P\n\016RegionSpaceUse\022)\n\013region_info\030\001 \001(\013" +
-      "2\024.hbase.pb.RegionInfo\022\023\n\013region_size\030\002 " +
-      "\001(\004\"J\n\033RegionSpaceUseReportRequest\022+\n\tsp" +
-      "ace_use\030\001 \003(\0132\030.hbase.pb.RegionSpaceUse\"" +
-      "\036\n\034RegionSpaceUseReportResponse2\227\005\n\031Regi" +
-      "onServerStatusService\022b\n\023RegionServerSta" +
-      "rtup\022$.hbase.pb.RegionServerStartupReque" +
-      "st\032%.hbase.pb.RegionServerStartupRespons",
-      "e\022_\n\022RegionServerReport\022#.hbase.pb.Regio" +
-      "nServerReportRequest\032$.hbase.pb.RegionSe" +
-      "rverReportResponse\022_\n\022ReportRSFatalError" +
-      "\022#.hbase.pb.ReportRSFatalErrorRequest\032$." +
-      "hbase.pb.ReportRSFatalErrorResponse\022q\n\030G" +
-      "etLastFlushedSequenceId\022).hbase.pb.GetLa" +
-      "stFlushedSequenceIdRequest\032*.hbase.pb.Ge" +
-      "tLastFlushedSequenceIdResponse\022z\n\033Report" +
-      "RegionStateTransition\022,.hbase.pb.ReportR" +
-      "egionStateTransitionRequest\032-.hbase.pb.R",
-      "eportRegionStateTransitionResponse\022e\n\024Re" +
-      "portRegionSpaceUse\022%.hbase.pb.RegionSpac" +
-      "eUseReportRequest\032&.hbase.pb.RegionSpace" +
-      "UseReportResponseBU\n1org.apache.hadoop.h" +
-      "base.shaded.protobuf.generatedB\030RegionSe" +
-      "rverStatusProtosH\001\210\001\001\240\001\001"
+      "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\

<TRUNCATED>

[12/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 78bd715..096172a 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,6 +30,7 @@ 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;
@@ -105,10 +106,7 @@ public class DeleteColumnFamilyProcedure
         setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
-          addChildProcedure(env.getAssignmentManager()
-            .createReopenProcedures(getRegionInfoList(env)));
-        }
+        reOpenAllRegionsIfTableIsOnline(env);
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -294,8 +292,7 @@ public class DeleteColumnFamilyProcedure
     env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    //reOpenAllRegionsIfTableIsOnline(env);
-    // TODO: NUKE ROLLBACK!!!!
+    reOpenAllRegionsIfTableIsOnline(env);
   }
 
   /**
@@ -319,6 +316,25 @@ 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
@@ -360,8 +376,7 @@ public class DeleteColumnFamilyProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = env.getAssignmentManager().getRegionStates()
-          .getRegionsOfTable(getTableName());
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 04dfc60..bda68eb 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,6 +44,7 @@ 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;
@@ -96,8 +97,8 @@ public class DeleteTableProcedure
           }
 
           // TODO: Move out... in the acquireLock()
-          LOG.debug("Waiting for '" + getTableName() + "' regions in transition");
-          regions = env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName());
+          LOG.debug("waiting for '" + getTableName() + "' regions in transition");
+          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 
@@ -349,7 +350,8 @@ 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) {
@@ -383,9 +385,11 @@ 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.");
-    env.getMasterServices().getAssignmentManager().deleteTable(tableName);
+    am.getRegionStates().tableDeleted(tableName);
 
     // If entry for this table states, remove it.
     LOG.debug("Marking '" + tableName + "' as deleted.");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 409ca26..b53ce45 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,9 +21,12 @@ 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;
@@ -31,11 +34,17 @@ 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
@@ -107,8 +116,12 @@ public class DisableTableProcedure
         setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
         break;
       case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
-        addChildProcedure(env.getAssignmentManager().createUnassignProcedures(tableName));
-        setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
+        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");
+        }
         break;
       case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
         setTableStateToDisabled(env, tableName);
@@ -236,7 +249,7 @@ public class DisableTableProcedure
       // set the state later on). A quick state check should be enough for us to move forward.
       TableStateManager tsm = env.getMasterServices().getTableStateManager();
       TableState.State state = tsm.getTableState(tableName);
-      if (!state.equals(TableState.State.ENABLED)){
+      if(!state.equals(TableState.State.ENABLED)){
         LOG.info("Table " + tableName + " isn't enabled;is "+state.name()+"; skipping disable");
         setFailure("master-disable-table", new TableNotEnabledException(
                 tableName+" state is "+state.name()));
@@ -277,6 +290,83 @@ 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
@@ -338,4 +428,64 @@ 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/a3c5a744/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
deleted file mode 100644
index 15ed429..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
+++ /dev/null
@@ -1,584 +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.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.REGION_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/a3c5a744/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 4f4b5b1..4d67edd 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,20 +21,34 @@ 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
@@ -100,7 +114,7 @@ public class EnableTableProcedure
         setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
         break;
       case ENABLE_TABLE_MARK_REGIONS_ONLINE:
-        addChildProcedure(env.getAssignmentManager().createAssignProcedures(tableName));
+        markRegionsOnline(env, tableName, true);
         setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLED_TABLE_STATE);
         break;
       case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
@@ -273,6 +287,137 @@ 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
@@ -312,6 +457,31 @@ 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/a3c5a744/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 31d05a7..4b9a7ab 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,19 +19,32 @@
 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
  */
@@ -47,13 +60,16 @@ public final class MasterDDLOperationHelper {
   public static void deleteColumnFamilyFromFileSystem(
       final MasterProcedureEnv env,
       final TableName tableName,
-      final List<HRegionInfo> regionInfoList,
+      List<HRegionInfo> regionInfoList,
       final byte[] familyName,
-      final boolean hasMob) throws IOException {
+      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);
@@ -65,4 +81,77 @@ 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/a3c5a744/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 f815bea..c21137d 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 = 16;
+  public static final int DEFAULT_MIN_MASTER_PROCEDURE_THREADS = 4;
 
   /**
    * Procedure replay sanity check. In case a WAL is missing or unreadable we

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 0f1c40f..2cd5b08 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,7 +32,6 @@ 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;
@@ -94,19 +93,12 @@ 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() {
@@ -125,10 +117,6 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.getConfiguration();
   }
 
-  public AssignmentManager getAssignmentManager() {
-    return master.getAssignmentManager();
-  }
-
   public MasterCoprocessorHost getMasterCoprocessorHost() {
     return master.getMasterCoprocessorHost();
   }
@@ -137,12 +125,7 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return procSched;
   }
 
-  public RSProcedureDispatcher getRemoteDispatcher() {
-    return remoteDispatcher;
-  }
-
   public boolean isRunning() {
-    if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
     return master.getMasterProcedureExecutor().isRunning();
   }
 
@@ -151,18 +134,11 @@ public class MasterProcedureEnv implements ConfigurationObserver {
   }
 
   public boolean waitInitialized(Procedure proc) {
-    return procSched.waitEvent(master.getInitializedEvent(), proc);
+    return procSched.waitEvent(((HMaster)master).getInitializedEvent(), proc);
   }
 
   public boolean waitServerCrashProcessingEnabled(Procedure proc) {
-    if (master instanceof HMaster) {
-      return procSched.waitEvent(((HMaster)master).getServerCrashProcessingEnabledEvent(), proc);
-    }
-    return false;
-  }
-
-  public boolean waitFailoverCleanup(Procedure proc) {
-    return procSched.waitEvent(master.getAssignmentManager().getFailoverCleanupEvent(), proc);
+    return procSched.waitEvent(((HMaster)master).getServerCrashProcessingEnabledEvent(), proc);
   }
 
   public void setEventReady(ProcedureEvent event, boolean isReady) {
@@ -177,4 +153,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/a3c5a744/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 1410748..15b557a 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
@@ -598,13 +598,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         return false;
       // region operations are using the shared-lock on the table
       // and then they will grab an xlock on the region.
-      case REGION_SPLIT:
-      case REGION_MERGE:
-      case REGION_ASSIGN:
-      case REGION_UNASSIGN:
+      case SPLIT:
+      case MERGE:
+      case ASSIGN:
+      case UNASSIGN:
       case REGION_EDIT:
-      case REGION_GC:
-      case MERGED_REGIONS_GC:
         return false;
       default:
         break;
@@ -817,11 +815,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       boolean hasLock = true;
       final LockAndQueue[] regionLocks = new LockAndQueue[regionInfo.length];
       for (int i = 0; i < regionInfo.length; ++i) {
-        LOG.info(procedure + " " + table + " " + regionInfo[i].getRegionNameAsString());
-        assert table != null;
-        assert regionInfo[i] != null;
-        assert regionInfo[i].getTable() != null;
-        assert regionInfo[i].getTable().equals(table): regionInfo[i] + " " + procedure;
+        assert regionInfo[i].getTable().equals(table);
         assert i == 0 || regionInfo[i] != regionInfo[i - 1] : "duplicate region: " + regionInfo[i];
 
         regionLocks[i] = locking.getRegionLock(regionInfo[i].getEncodedName());
@@ -1260,12 +1254,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     */
   @VisibleForTesting
   public String dumpLocks() throws IOException {
-    schedLock();
-    try {
-      // TODO: Refactor so we stream out locks for case when millions; i.e. take a PrintWriter
-      return this.locking.toString();
-    } finally {
-      schedUnlock();
-    }
+    // TODO: Refactor so we stream out locks for case when millions; i.e. take a PrintWriter
+    return this.locking.toString();
   }
 }


[14/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 2b1de9d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ /dev/null
@@ -1,776 +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.assignment;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-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.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.MasterSwitchType;
-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.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.procedure2.ProcedureSuspendedException;
-import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-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.AdminProtos.GetRegionInfoResponse;
-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;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.lmax.disruptor.YieldingWaitStrategy;
-
-/**
- * The procedure to Merge a region in a table.
- * This procedure takes an exclusive table lock since it is working over multiple regions.
- * It holds the lock for the life of the procedure.
- */
-@InterfaceAudience.Private
-public class MergeTableRegionsProcedure
-    extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
-  private static final Log LOG = LogFactory.getLog(MergeTableRegionsProcedure.class);
-  private Boolean traceEnabled;
-  private volatile boolean lock = false;
-  private ServerName regionLocation;
-  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;
-  }
-
-  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 ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(this + " execute state=" + state);
-    }
-    try {
-      switch (state) {
-      case MERGE_TABLE_REGIONS_PREPARE:
-        if (!prepareMergeRegion(env)) {
-          assert isFailed() : "Merge region should have an exception here";
-          return Flow.NO_MORE_STATE;
-        }
-        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 " + HRegionInfo.getShortNameToLog(regionsToMerge) +
-        " 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;
-      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 "
-          + HRegionInfo.getShortNameToLog(regionsToMerge) + " 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.forNumber(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(HRegionInfo.getShortNameToLog(regionsToMerge));
-    sb.append(", forcibly=");
-    sb.append(forcible);
-  }
-
-  @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])) {
-      try {
-        LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks());
-      } catch (IOException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    this.lock = true;
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    this.lock = false;
-    env.getProcedureScheduler().wakeRegions(this, getTableName(),
-      mergedRegion, regionsToMerge[0], regionsToMerge[1]);
-  }
-
-  @Override
-  protected boolean holdLock(MasterProcedureEnv env) {
-    return true;
-  }
-
-  @Override
-  protected boolean hasLock(MasterProcedureEnv env) {
-    return this.lock;
-  }
-
-  @Override
-  public TableName getTableName() {
-    return mergedRegion.getTable();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.REGION_MERGE;
-  }
-
-  /**
-   * Prepare merge and do some check
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private boolean 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 " + HRegionInfo.getShortNameToLog(regionsToMerge) +
-        ", 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);
-    }
-
-    if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
-      String regionsStr = Arrays.deepToString(regionsToMerge);
-      LOG.warn("merge switch is off! skip merge of " + regionsStr);
-      super.setFailure(getClass().getSimpleName(),
-          new IOException("Merge of " + regionsStr + " failed because merge switch is off"));
-      return false;
-    }
-    
-
-    // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it
-    // along w/ the failure so can see why we are not mergeable at this time.
-    IOException mergeableCheckIOE = null;
-    boolean mergeable = false;
-    RegionState current = regionStateA;
-    try {
-      mergeable = isMergeable(env, current);
-    } catch (IOException e) {
-      mergeableCheckIOE = e;
-    }
-    if (mergeable && mergeableCheckIOE == null) {
-      current = regionStateB;
-      try {
-        mergeable = isMergeable(env, current);
-      } catch (IOException e) {
-        mergeableCheckIOE = e;
-      }
-    }
-    if (!mergeable) {
-      IOException e = new IOException(current.getRegion().getShortNameToLog() + " NOT mergeable");
-      if (mergeableCheckIOE != null) e.initCause(mergeableCheckIOE);
-      super.setFailure(getClass().getSimpleName(), e);
-      return false;
-    }
-
-    return true;
-  }
-
-  private boolean isMergeable(final MasterProcedureEnv env, final RegionState rs)
-  throws IOException {
-    GetRegionInfoResponse response =
-      Util.getRegionInfoResponse(env, rs.getServerName(), rs.getRegion());
-    return response.hasSplittable() && response.getSplittable();
-  }
-
-  /**
-   * 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 " + HRegionInfo.getShortNameToLog(regionsToMerge) +
-          " merge.");
-      }
-    }
-    // TODO: Clean up split and merge. Currently all over the place.
-    env.getMasterServices().getMasterQuotaManager().onRegionMerged(this.mergedRegion);
-  }
-
-  /**
-   * 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 " + HRegionInfo.getShortNameToLog(regionsToMerge) +
-          " 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.
-   */
-  private void updateMetaForMergedRegions(final MasterProcedureEnv env)
-  throws IOException, ProcedureYieldException {
-    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]);
-      // May still be null here but return null and let caller deal.
-      // Means we lost the in-memory-only location. We are in recovery
-      // or so. The caller should be able to deal w/ a null ServerName.
-      // Let them go to the Balancer to find one to use instead.
-    }
-    return regionLocation;
-  }
-
-  /**
-   * 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;
-  }
-
-  /**
-   * @return The merged region. Maybe be null if called to early or we failed.
-   */
-  @VisibleForTesting
-  public HRegionInfo getMergedRegion() {
-    return this.mergedRegion;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index d8c1b7d..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MoveRegionProcedure.java
+++ /dev/null
@@ -1,145 +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.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.RegionPlan;
-import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-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. It takes a lock on the region being moved.
- * It holds the lock for the life of the procedure.
- */
-@InterfaceAudience.Private
-public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<MoveRegionState> {
-  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 MasterProcedureEnv env, final RegionPlan plan) {
-    super(env, plan.getRegionInfo());
-    assert plan.getDestination() != null: plan.toString();
-    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.getSource(), 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(" ");
-    sb.append(plan);
-  }
-
-  @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()
-        // No need to serialize the HRegionInfo. The super class has the region.
-        .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 = getRegion(); // Get it from super class deserialization.
-    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/a3c5a744/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
deleted file mode 100644
index 21e0d9c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ /dev/null
@@ -1,327 +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.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 hbase:meta entry region=%s regionState=%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,
-      final long pid)
-      throws IOException {
-    if (regionInfo.isMetaRegion()) {
-      updateMetaLocation(regionInfo, regionLocation);
-    } else {
-      updateUserRegionLocation(regionInfo, state, regionLocation, lastHost, openSeqNum, pid);
-    }
-  }
-
-  public void updateRegionState(final long openSeqNum, final long pid,
-      final RegionState newState, final RegionState oldState) throws IOException {
-    updateRegionLocation(newState.getRegion(), newState.getState(), newState.getServerName(),
-        oldState != null ? oldState.getServerName() : null, openSeqNum, pid);
-  }
-
-  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,
-      final long pid)
-      throws IOException {
-    final int replicaId = regionInfo.getReplicaId();
-    final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo));
-    MetaTableAccessor.addRegionInfo(put, regionInfo);
-    final StringBuilder info = new StringBuilder("pid=" + pid + " updating hbase:meta row=");
-    info.append(regionInfo.getRegionNameAsString()).append(", regionState=").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(", regionLocation=").append(regionLocation);
-    } else if (regionLocation != null && !regionLocation.equals(lastHost)) {
-      // Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients
-      // currently; they want a server to hit. TODO: Make clients wait if no location.
-      put.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
-          Bytes.toBytes(regionLocation.getServerName()));
-      info.append(", regionLocation=").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 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/a3c5a744/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
deleted file mode 100644
index 082e171..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ /dev/null
@@ -1,969 +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.assignment;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-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.exceptions.UnexpectedStateException;
-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_EXPECTED_ON_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_EXPECTED_ON_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);
-    }
-  }
-
-  /**
-   * Current Region State.
-   * In-memory only. Not persisted.
-   */
-  // Mutable/Immutable? Changes have to be synchronized or not?
-  // Data members are volatile which seems to say multi-threaded access is fine.
-  // In the below we do check and set but the check state could change before
-  // we do the set because no synchronization....which seems dodgy. Clear up
-  // understanding here... how many threads accessing? Do locks make it so one
-  // thread at a time working on a single Region's RegionStateNode? Lets presume
-  // so for now. Odd is that elsewhere in this RegionStates, we synchronize on
-  // the RegionStateNode instance. TODO.
-  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;
-    /**
-     * A Region-in-Transition (RIT) moves through states.
-     * See {@link State} for complete list. A Region that
-     * is opened moves from OFFLINE => OPENING => OPENED.
-     */
-    private volatile State state = State.OFFLINE;
-
-    /**
-     * Updated whenever a call to {@link #setRegionLocation(ServerName)}
-     * or {@link #setState(State, State...)}.
-     */
-    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;
-        this.lastUpdate = EnvironmentEdgeManager.currentTime();
-      }
-      return expectedState;
-    }
-
-    /**
-     * Put region into OFFLINE mode (set state and clear location).
-     * @return Last recorded server deploy
-     */
-    public ServerName offline() {
-      setState(State.OFFLINE);
-      return setRegionLocation(null);
-    }
-
-    /**
-     * Set new {@link State} but only if currently in <code>expected</code> State
-     * (if not, throw {@link UnexpectedStateException}.
-     */
-    public State transitionState(final State update, final State... expected)
-    throws UnexpectedStateException {
-      if (!setState(update, expected)) {
-        throw new UnexpectedStateException("Expected " + Arrays.toString(expected) +
-          " so could move to " + update + " but current state=" + getState());
-      }
-      return update;
-    }
-
-    public boolean isInState(final State... expected) {
-      if (expected != null && expected.length > 0) {
-        boolean expectedState = false;
-        for (int i = 0; i < expected.length; ++i) {
-          expectedState |= (getState() == 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 ServerName setRegionLocation(final ServerName serverName) {
-      ServerName lastRegionLocation = this.regionLocation;
-      if (LOG.isTraceEnabled() && serverName == null) {
-        LOG.trace("Tracking when we are set to null " + this, new Throwable("TRACE"));
-      }
-      this.regionLocation = serverName;
-      this.lastUpdate = EnvironmentEdgeManager.currentTime();
-      return lastRegionLocation;
-    }
-
-    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 toShortString() {
-      // rit= is the current Region-In-Transition State -- see State enum.
-      return String.format("rit=%s, location=%s", getState(), getRegionLocation());
-    }
-
-    public String toDescriptiveString() {
-      return String.format("%s, table=%s, region=%s",
-        toShortString(), getTable(), getRegionInfo().getEncodedName());
-    }
-  }
-
-  // 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
-  /**
-   * RegionName -- i.e. HRegionInfo.getRegionName() -- as bytes to {@link RegionStateNode}
-   */
-  private final ConcurrentSkipListMap<byte[], RegionStateNode> regionsMap =
-      new ConcurrentSkipListMap<byte[], RegionStateNode>(Bytes.BYTES_COMPARATOR);
-
-  private final ConcurrentSkipListMap<HRegionInfo, RegionStateNode> regionInTransition =
-    new ConcurrentSkipListMap<HRegionInfo, RegionStateNode>();
-
-  /**
-   * Regions marked as offline on a read of hbase:meta. Unused or at least, once
-   * offlined, regions have no means of coming on line again. TODO.
-   */
-  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);
-  }
-
-  RegionStateNode getRegionNodeFromName(final byte[] regionName) {
-    return regionsMap.get(regionName);
-  }
-
-  protected RegionStateNode getRegionNode(final HRegionInfo regionInfo) {
-    return getRegionNodeFromName(regionInfo.getRegionName());
-  }
-
-  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());
-    // Remove from the offline regions map too if there.
-    if (this.regionOffline.containsKey(regionInfo)) {
-      if (LOG.isTraceEnabled()) LOG.trace("Removing from regionOffline Map: " + regionInfo);
-      this.regionOffline.remove(regionInfo);
-    }
-  }
-
-  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;
-  }
-
-  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;
-  }
-
-  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;
-  }
-
-  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).isEmpty();
-  }
-
-  public List<HRegionInfo> getRegionsOfTable(final TableName table) {
-    return getRegionsOfTable(table, false);
-  }
-
-  List<HRegionInfo> getRegionsOfTable(final TableName table, final boolean offline) {
-    final ArrayList<RegionStateNode> nodes = getTableRegionStateNodes(table);
-    final ArrayList<HRegionInfo> hris = new ArrayList<HRegionInfo>(nodes.size());
-    for (RegionStateNode node: nodes) {
-      if (include(node, offline)) hris.add(node.getRegionInfo());
-    }
-    return hris;
-  }
-
-  /**
-   * Utility. Whether to include region in list of regions. Default is to
-   * weed out split and offline regions.
-   * @return True if we should include the <code>node</code> (do not include
-   * if split or offline unless <code>offline</code> is set to true.
-   */
-  boolean include(final RegionStateNode node, final boolean offline) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("WORKING ON " + node + " " + node.getRegionInfo());
-    }
-    if (node.isInState(State.SPLIT)) return false;
-    if (node.isInState(State.OFFLINE) && !offline) return false;
-    final HRegionInfo hri = node.getRegionInfo();
-    return (!hri.isOffline() && !hri.isSplit()) ||
-        ((hri.isOffline() || hri.isSplit()) && offline);
-  }
-
-  /**
-   * 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);
-      /* THIS HAS TO BE WRONG. THIS IS SPLITTING OF REGION, NOT SPLITTING WALs.
-      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);
-  }
-
-  /**
-   * @return True if region is offline (In OFFLINE or CLOSED state).
-   */
-  public boolean isRegionOffline(final HRegionInfo regionInfo) {
-    return isRegionInState(regionInfo, State.OFFLINE, State.CLOSED);
-  }
-
-  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<>();
-    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();
-      if (serverName == null) {
-        LOG.info("Skipping, no server for " + node);
-        continue;
-      }
-      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;
-  }
-
-  /**
-   * @return If a procedure-in-transition for <code>hri</code>, return it else null.
-   */
-  public RegionTransitionProcedure getRegionTransitionProcedure(final HRegionInfo hri) {
-    RegionStateNode node = regionInTransition.get(hri);
-    if (node == null) return null;
-    return node.getProcedure();
-  }
-
-  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
-  // ==========================================================================
-  // TODO: Populated when we read meta but regions never make it out of here.
-  public void addToOfflineRegions(final RegionStateNode regionNode) {
-    LOG.info("Added to offline, CURRENTLY NEVER CLEARED!!! " + regionNode);
-    regionOffline.put(regionNode.getRegionInfo(), regionNode);
-  }
-
-  // TODO: Unused.
-  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()) {
-      totalLoad += node.getRegionCount();
-      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();
-  }
-}


[05/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 59e8fb3..32bce26 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
@@ -27,11 +27,8 @@ import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.SortedMap;
-import java.util.SortedSet;
 import java.util.TreeMap;
-import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,141 +36,267 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CategoryBasedTimeout;
+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.MetaMockingUtil;
 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.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+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.CatalogJanitor.SplitParentFirstComparator;
-import org.apache.hadoop.hbase.master.assignment.MockMasterServices;
-import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+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.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
+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.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Triple;
-import org.junit.After;
-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;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category({MasterTests.class, SmallTests.class})
 public class TestCatalogJanitor {
   private static final Log LOG = LogFactory.getLog(TestCatalogJanitor.class);
-  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
-     withTimeout(this.getClass()).withLookingForStuckThread(true).build();
-  @Rule public final TestName name = new TestName();
-  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
-  private MockMasterServices masterServices;
-  private CatalogJanitor janitor;
+
+  @Rule
+  public TestName name = new TestName();
 
   @BeforeClass
-  public static void beforeClass() throws Exception {
+  public static void setup() throws Exception {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
   }
+  /**
+   * Mock MasterServices for tests below.
+   */
+  class MockMasterServices extends MockNoopMasterServices {
+    private final ClusterConnection connection;
+    private final MasterFileSystem mfs;
+    private final AssignmentManager asm;
+    private final ServerManager sm;
+
+    MockMasterServices(final HBaseTestingUtility htu) throws IOException {
+      super(htu.getConfiguration());
+
+      ClientProtos.ClientService.BlockingInterface ri =
+        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+      MutateResponse.Builder builder = MutateResponse.newBuilder();
+      builder.setProcessed(true);
+      try {
+        Mockito.when(ri.mutate(
+          (RpcController)Mockito.any(), (MutateRequest)Mockito.any())).
+            thenReturn(builder.build());
+      } catch (ServiceException se) {
+        throw ProtobufUtil.handleRemoteException(se);
+      }
+      try {
+        Mockito.when(ri.multi(
+          (RpcController)Mockito.any(), (MultiRequest)Mockito.any())).
+            thenAnswer(new Answer<MultiResponse>() {
+              @Override
+              public MultiResponse answer(InvocationOnMock invocation) throws Throwable {
+                return buildMultiResponse( (MultiRequest)invocation.getArguments()[1]);
+              }
+            });
+      } catch (ServiceException se) {
+        throw ProtobufUtil.getRemoteException(se);
+      }
+      // Mock an ClusterConnection and a AdminProtocol implementation.  Have the
+      // ClusterConnection return the HRI.  Have the HRI return a few mocked up responses
+      // to make our test work.
+      this.connection =
+        HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
+          Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri,
+            ServerName.valueOf("example.org,12345,6789"),
+          HRegionInfo.FIRST_META_REGIONINFO);
+      // Set hbase.rootdir into test dir.
+      FileSystem.get(getConfiguration());
+      Path rootdir = FSUtils.getRootDir(getConfiguration());
+      FSUtils.setRootDir(getConfiguration(), rootdir);
+      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+
+      this.mfs = new MasterFileSystem(this);
+      this.asm = Mockito.mock(AssignmentManager.class);
+      this.sm = Mockito.mock(ServerManager.class);
+    }
 
-  @Before
-  public void setup() throws IOException {
-    setRootDirAndCleanIt(HTU, this.name.getMethodName());
-    NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers =
-        new ConcurrentSkipListMap<ServerName, SortedSet<byte []>>();
-    this.masterServices =
-        new MockMasterServices(HTU.getConfiguration(), regionsToRegionServers);
-    this.masterServices.start(10, null);
-    this.janitor = new CatalogJanitor(masterServices);
-  }
+    @Override
+    public AssignmentManager getAssignmentManager() {
+      return this.asm;
+    }
+
+    @Override
+    public MasterFileSystem getMasterFileSystem() {
+      return this.mfs;
+    }
+
+    @Override
+    public ClusterConnection getConnection() {
+      return this.connection;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return ServerName.valueOf("mockserver.example.org", 1234, -1L);
+    }
+
+    @Override
+    public ServerManager getServerManager() {
+      return this.sm;
+    }
+
+    @Override
+    public CoordinatedStateManager getCoordinatedStateManager() {
+      BaseCoordinatedStateManager m = Mockito.mock(BaseCoordinatedStateManager.class);
+      SplitLogManagerCoordination c = Mockito.mock(SplitLogManagerCoordination.class);
+      Mockito.when(m.getSplitLogManagerCoordination()).thenReturn(c);
+      SplitLogManagerDetails d = Mockito.mock(SplitLogManagerDetails.class);
+      Mockito.when(c.getDetails()).thenReturn(d);
+      return m;
+    }
 
-  @After
-  public void teardown() {
-    this.janitor.cancel(true);
-    this.masterServices.stop("DONE");
+    @Override
+    public TableDescriptors getTableDescriptors() {
+      return new TableDescriptors() {
+        @Override
+        public HTableDescriptor remove(TableName tablename) throws IOException {
+          // noop
+          return null;
+        }
+
+        @Override
+        public Map<String, HTableDescriptor> getAll() throws IOException {
+          // noop
+          return null;
+        }
+
+        @Override public Map<String, HTableDescriptor> getAllDescriptors() throws IOException {
+          // noop
+          return null;
+        }
+
+        @Override
+        public HTableDescriptor get(TableName tablename)
+            throws IOException {
+          return createHTableDescriptor();
+        }
+
+        @Override
+        public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
+          return null;
+        }
+
+        @Override
+        public void add(HTableDescriptor htd) throws IOException {
+          // noop
+        }
+
+        @Override
+        public void setCacheOn() throws IOException {
+        }
+
+        @Override
+        public void setCacheOff() throws IOException {
+        }
+      };
+    }
   }
 
-  /**
-   * Test clearing a split parent.
-   */
   @Test
   public void testCleanParent() throws IOException, InterruptedException {
-    TableDescriptor td = createTableDescriptorForCurrentMethod();
-    // Create regions.
-    HRegionInfo parent =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
-    // Test that when both daughter regions are in place, that we do not remove the parent.
-    Result r = createResult(parent, splita, splitb);
-    // Add a reference under splitA directory so we don't clear out the parent.
-    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
-    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
-    Path parentdir = new Path(tabledir, parent.getEncodedName());
-    Path storedir = HStore.getStoreHomedir(tabledir, splita, td.getColumnFamilies()[0].getName());
-    Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
-    long now = System.currentTimeMillis();
-    // Reference name has this format: StoreFile#REF_NAME_PARSER
-    Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
-    FileSystem fs = this.masterServices.getMasterFileSystem().getFileSystem();
-    Path path = ref.write(fs, p);
-    assertTrue(fs.exists(path));
-    LOG.info("Created reference " + path);
-    // Add a parentdir for kicks so can check it gets removed by the catalogjanitor.
-    fs.mkdirs(parentdir);
-    assertFalse(this.janitor.cleanParent(parent, r));
-    ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
-    assertTrue(fs.exists(parentdir));
-    // Remove the reference file and try again.
-    assertTrue(fs.delete(p, true));
-    assertTrue(this.janitor.cleanParent(parent, r));
-    // Parent cleanup is run async as a procedure. Make sure parentdir is removed.
-    ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
-    assertTrue(!fs.exists(parentdir));
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    setRootDirAndCleanIt(htu, "testCleanParent");
+    MasterServices services = new MockMasterServices(htu);
+    try {
+      CatalogJanitor janitor = new CatalogJanitor(services);
+      // Create regions.
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+      htd.addFamily(new HColumnDescriptor("f"));
+      HRegionInfo parent =
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+            Bytes.toBytes("eee"));
+      HRegionInfo splita =
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+            Bytes.toBytes("ccc"));
+      HRegionInfo splitb =
+        new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
+            Bytes.toBytes("eee"));
+      // Test that when both daughter regions are in place, that we do not
+      // remove the parent.
+      Result r = createResult(parent, splita, splitb);
+      // Add a reference under splitA directory so we don't clear out the parent.
+      Path rootdir = services.getMasterFileSystem().getRootDir();
+      Path tabledir =
+        FSUtils.getTableDir(rootdir, htd.getTableName());
+      Path storedir = HStore.getStoreHomedir(tabledir, splita,
+          htd.getColumnFamilies()[0].getName());
+      Reference ref = Reference.createTopReference(Bytes.toBytes("ccc"));
+      long now = System.currentTimeMillis();
+      // Reference name has this format: StoreFile#REF_NAME_PARSER
+      Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
+      FileSystem fs = services.getMasterFileSystem().getFileSystem();
+      Path path = ref.write(fs, p);
+      assertTrue(fs.exists(path));
+      assertFalse(janitor.cleanParent(parent, r));
+      // Remove the reference file and try again.
+      assertTrue(fs.delete(p, true));
+      assertTrue(janitor.cleanParent(parent, r));
+    } finally {
+      services.stop("shutdown");
+    }
   }
 
   /**
    * Make sure parent gets cleaned up even if daughter is cleaned up before it.
+   * @throws IOException
+   * @throws InterruptedException
    */
   @Test
   public void testParentCleanedEvenIfDaughterGoneFirst()
   throws IOException, InterruptedException {
-    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(this.name.getMethodName(),
-        Bytes.toBytes("eee"));
+    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
+      "testParentCleanedEvenIfDaughterGoneFirst", Bytes.toBytes("eee"));
   }
 
   /**
    * Make sure last parent with empty end key gets cleaned up even if daughter is cleaned up before it.
+   * @throws IOException
+   * @throws InterruptedException
    */
   @Test
   public void testLastParentCleanedEvenIfDaughterGoneFirst()
   throws IOException, InterruptedException {
-    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(this.name.getMethodName(),
-        new byte[0]);
-  }
-
-  /**
-   * @return A TableDescriptor with a tableName of current method name and a column
-   * family that is MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)
-   */
-  private TableDescriptor createTableDescriptorForCurrentMethod() {
-    return TableDescriptorBuilder.newBuilder(TableName.valueOf(this.name.getMethodName())).
-        addFamily(new HColumnDescriptor(MockMasterServices.DEFAULT_COLUMN_FAMILY_NAME)).
-        build();
+    parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
+      "testLastParentCleanedEvenIfDaughterGoneFirst", new byte[0]);
   }
 
   /**
@@ -181,40 +304,50 @@ public class TestCatalogJanitor {
    *
    * @param rootDir the test case name, used as the HBase testing utility root
    * @param lastEndKey the end key of the split parent
+   * @throws IOException
+   * @throws InterruptedException
    */
   private void parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
   final String rootDir, final byte[] lastEndKey)
   throws IOException, InterruptedException {
-    TableDescriptor td = createTableDescriptorForCurrentMethod();
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    setRootDirAndCleanIt(htu, rootDir);
+    MasterServices services = new MockMasterServices(htu);
+    CatalogJanitor janitor = new CatalogJanitor(services);
+    final HTableDescriptor htd = createHTableDescriptor();
+
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
-    HRegionInfo parent = new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), lastEndKey);
+
+    // Parent
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      lastEndKey);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      Bytes.toBytes("ccc"));
     Thread.sleep(1001);
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("bbb"));
-    HRegionInfo splitab =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("bbb"), Bytes.toBytes("ccc"));
+    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      Bytes.toBytes("bbb"));
+    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), Bytes.toBytes("bbb"),
+      Bytes.toBytes("ccc"));
 
     // Daughter b
-    HRegionInfo splitb =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), lastEndKey);
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
+      lastEndKey);
     Thread.sleep(1001);
     // Make Daughters of daughterb; splitba and splitbb.
-    HRegionInfo splitba =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), Bytes.toBytes("ddd"));
-    HRegionInfo splitbb =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("ddd"), lastEndKey);
+    HRegionInfo splitba = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
+      Bytes.toBytes("ddd"));
+    HRegionInfo splitbb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ddd"),
+    lastEndKey);
 
     // First test that our Comparator works right up in CatalogJanitor.
-    SortedMap<HRegionInfo, Result> regions =
-        new TreeMap<>(new CatalogJanitor.SplitParentFirstComparator());
+    // Just fo kicks.
+    SortedMap<HRegionInfo, Result> regions = new TreeMap<>(new CatalogJanitor.SplitParentFirstComparator());
     // Now make sure that this regions map sorts as we expect it to.
     regions.put(parent, createResult(parent, splita, splitb));
     regions.put(splitb, createResult(splitb, splitba, splitbb));
@@ -232,9 +365,10 @@ public class TestCatalogJanitor {
       index++;
     }
 
-    // Now play around with the cleanParent function. Create a ref from splita up to the parent.
+    // Now play around with the cleanParent function.  Create a ref from splita
+    // up to the parent.
     Path splitaRef =
-        createReferences(this.masterServices, td, parent, splita, Bytes.toBytes("ccc"), false);
+      createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), false);
     // Make sure actual super parent sticks around because splita has a ref.
     assertFalse(janitor.cleanParent(parent, regions.get(parent)));
 
@@ -246,13 +380,13 @@ public class TestCatalogJanitor {
     // the daughter splita can be split (can't split if still references).
     // BUT make the timing such that the daughter gets cleaned up before we
     // can get a chance to let go of the parent.
-    FileSystem fs = FileSystem.get(HTU.getConfiguration());
+    FileSystem fs = FileSystem.get(htu.getConfiguration());
     assertTrue(fs.delete(splitaRef, true));
     // Create the refs from daughters of splita.
     Path splitaaRef =
-      createReferences(this.masterServices, td, splita, splitaa, Bytes.toBytes("bbb"), false);
+      createReferences(services, htd, splita, splitaa, Bytes.toBytes("bbb"), false);
     Path splitabRef =
-      createReferences(this.masterServices, td, splita, splitab, Bytes.toBytes("bbb"), true);
+      createReferences(services, htd, splita, splitab, Bytes.toBytes("bbb"), true);
 
     // Test splita.  It should stick around because references from splitab, etc.
     assertFalse(janitor.cleanParent(splita, regions.get(splita)));
@@ -264,67 +398,74 @@ public class TestCatalogJanitor {
 
     // Super parent should get cleaned up now both splita and splitb are gone.
     assertTrue(janitor.cleanParent(parent, regions.get(parent)));
+
+    services.stop("test finished");
+    janitor.cancel(true);
   }
 
   /**
    * CatalogJanitor.scan() should not clean parent regions if their own
-   * parents are still referencing them. This ensures that grandparent regions
+   * parents are still referencing them. This ensures that grandfather regions
    * do not point to deleted parent regions.
    */
   @Test
   public void testScanDoesNotCleanRegionsWithExistingParents() throws Exception {
-    TableDescriptor td = createTableDescriptorForCurrentMethod();
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    setRootDirAndCleanIt(htu, "testScanDoesNotCleanRegionsWithExistingParents");
+    MasterServices services = new MockMasterServices(htu);
+
+    final HTableDescriptor htd = createHTableDescriptor();
+
     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
 
     // Parent
-    HRegionInfo parent = new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"),
-            HConstants.EMPTY_BYTE_ARRAY, true);
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      new byte[0], true);
     // Sleep a second else the encoded name on these regions comes out
     // same for all with same start key and made in same second.
     Thread.sleep(1001);
 
     // Daughter a
-    HRegionInfo splita =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"), true);
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      Bytes.toBytes("ccc"), true);
     Thread.sleep(1001);
-
     // Make daughters of daughter a; splitaa and splitab.
-    HRegionInfo splitaa =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("aaa"), Bytes.toBytes("bbb"), false);
-    HRegionInfo splitab =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), false);
+    HRegionInfo splitaa = new HRegionInfo(htd.getTableName(), Bytes.toBytes("aaa"),
+      Bytes.toBytes("bbb"), false);
+    HRegionInfo splitab = new HRegionInfo(htd.getTableName(), Bytes.toBytes("bbb"),
+      Bytes.toBytes("ccc"), false);
 
     // Daughter b
-    HRegionInfo splitb =
-        new HRegionInfo(td.getTableName(), Bytes.toBytes("ccc"), HConstants.EMPTY_BYTE_ARRAY);
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(), Bytes.toBytes("ccc"),
+        new byte[0]);
     Thread.sleep(1001);
 
-    // Parent has daughters splita and splitb. Splita has daughters splitaa and splitab.
     final Map<HRegionInfo, Result> splitParents = new TreeMap<>(new SplitParentFirstComparator());
     splitParents.put(parent, createResult(parent, splita, splitb));
     splita.setOffline(true); //simulate that splita goes offline when it is split
-    splitParents.put(splita, createResult(splita, splitaa, splitab));
+    splitParents.put(splita, createResult(splita, splitaa,splitab));
 
     final Map<HRegionInfo, Result> mergedRegions = new TreeMap<>();
-    CatalogJanitor spy = spy(this.janitor);
-    doReturn(new Triple<>(10, mergedRegions, splitParents)).when(spy).
-      getMergedRegionsAndSplitParents();
+    CatalogJanitor janitor = spy(new CatalogJanitor(services));
+    doReturn(new Triple<>(10, mergedRegions, splitParents)).when(janitor)
+        .getMergedRegionsAndSplitParents();
 
-    // Create ref from splita to parent
-    LOG.info("parent=" + parent.getShortNameToLog() + ", splita=" + splita.getShortNameToLog());
+    //create ref from splita to parent
     Path splitaRef =
-        createReferences(this.masterServices, td, parent, splita, Bytes.toBytes("ccc"), false);
-    LOG.info("Created reference " + splitaRef);
+        createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), false);
 
-    // Parent and splita should not be removed because a reference from splita to parent.
-    assertEquals(0, spy.scan());
+    //parent and A should not be removed
+    assertEquals(0, janitor.scan());
 
-    // Now delete the ref
-    FileSystem fs = FileSystem.get(HTU.getConfiguration());
+    //now delete the ref
+    FileSystem fs = FileSystem.get(htu.getConfiguration());
     assertTrue(fs.delete(splitaRef, true));
 
     //now, both parent, and splita can be deleted
-    assertEquals(2, spy.scan());
+    assertEquals(2, janitor.scan());
+
+    services.stop("test finished");
+    janitor.cancel(true);
   }
 
   /**
@@ -334,7 +475,7 @@ public class TestCatalogJanitor {
   @Test
   public void testSplitParentFirstComparator() {
     SplitParentFirstComparator comp = new SplitParentFirstComparator();
-    TableDescriptor td = createTableDescriptorForCurrentMethod();
+    final HTableDescriptor htd = createHTableDescriptor();
 
     /*  Region splits:
      *
@@ -355,12 +496,15 @@ public class TestCatalogJanitor {
      */
 
     // root region
-    HRegionInfo rootRegion = new HRegionInfo(td.getTableName(),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, true);
-    HRegionInfo firstRegion = new HRegionInfo(td.getTableName(),
-      HConstants.EMPTY_START_ROW, Bytes.toBytes("bbb"), true);
-    HRegionInfo lastRegion = new HRegionInfo(td.getTableName(),
-      Bytes.toBytes("bbb"), HConstants.EMPTY_END_ROW, true);
+    HRegionInfo rootRegion = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
+      HConstants.EMPTY_END_ROW, true);
+    HRegionInfo firstRegion = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
+      Bytes.toBytes("bbb"), true);
+    HRegionInfo lastRegion = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
+      HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(rootRegion, rootRegion) == 0);
     assertTrue(comp.compare(firstRegion, firstRegion) == 0);
@@ -370,15 +514,19 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegion, lastRegion) < 0);
 
     //first region split into a, b
-    HRegionInfo firstRegiona = new HRegionInfo(td.getTableName(),
-      HConstants.EMPTY_START_ROW, Bytes.toBytes("aaa"), true);
-    HRegionInfo firstRegionb = new HRegionInfo(td.getTableName(),
-        Bytes.toBytes("aaa"), Bytes.toBytes("bbb"), true);
+    HRegionInfo firstRegiona = new HRegionInfo(htd.getTableName(),
+      HConstants.EMPTY_START_ROW,
+      Bytes.toBytes("aaa"), true);
+    HRegionInfo firstRegionb = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("aaa"),
+      Bytes.toBytes("bbb"), true);
     //last region split into a, b
-    HRegionInfo lastRegiona = new HRegionInfo(td.getTableName(),
-      Bytes.toBytes("bbb"), Bytes.toBytes("ddd"), true);
-    HRegionInfo lastRegionb = new HRegionInfo(td.getTableName(),
-      Bytes.toBytes("ddd"), HConstants.EMPTY_END_ROW, true);
+    HRegionInfo lastRegiona = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
+      Bytes.toBytes("ddd"), true);
+    HRegionInfo lastRegionb = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("ddd"),
+      HConstants.EMPTY_END_ROW, true);
 
     assertTrue(comp.compare(firstRegiona, firstRegiona) == 0);
     assertTrue(comp.compare(firstRegionb, firstRegionb) == 0);
@@ -401,47 +549,58 @@ public class TestCatalogJanitor {
     assertTrue(comp.compare(firstRegionb, lastRegiona) < 0);
     assertTrue(comp.compare(firstRegionb, lastRegionb) < 0);
 
-    HRegionInfo lastRegionaa = new HRegionInfo(td.getTableName(),
-      Bytes.toBytes("bbb"), Bytes.toBytes("ccc"), false);
-    HRegionInfo lastRegionab = new HRegionInfo(td.getTableName(),
-      Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), false);
+    HRegionInfo lastRegionaa = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("bbb"),
+      Bytes.toBytes("ccc"), false);
+    HRegionInfo lastRegionab = new HRegionInfo(htd.getTableName(),
+      Bytes.toBytes("ccc"),
+      Bytes.toBytes("ddd"), false);
 
     assertTrue(comp.compare(lastRegiona, lastRegionaa) < 0);
     assertTrue(comp.compare(lastRegiona, lastRegionab) < 0);
     assertTrue(comp.compare(lastRegionaa, lastRegionab) < 0);
+
   }
 
   @Test
   public void testArchiveOldRegion() throws Exception {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    setRootDirAndCleanIt(htu, "testCleanParent");
+    MasterServices services = new MockMasterServices(htu);
+
+    // create the janitor
+    CatalogJanitor janitor = new CatalogJanitor(services);
+
     // Create regions.
-    TableDescriptor td = createTableDescriptorForCurrentMethod();
-    HRegionInfo parent = new HRegionInfo(td.getTableName(),
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+    htd.addFamily(new HColumnDescriptor("f"));
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(td.getTableName(),
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(td.getTableName(),
-        Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
+        Bytes.toBytes("ccc"),
+        Bytes.toBytes("eee"));
 
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     Result parentMetaRow = createResult(parent, splita, splitb);
-    FileSystem fs = FileSystem.get(HTU.getConfiguration());
-    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
+    FileSystem fs = FileSystem.get(htu.getConfiguration());
+    Path rootdir = services.getMasterFileSystem().getRootDir();
     // have to set the root directory since we use it in HFileDisposer to figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
-    Path tabledir = FSUtils.getTableDir(rootdir, td.getTableName());
-    Path storedir = HStore.getStoreHomedir(tabledir, parent, td.getColumnFamilies()[0].getName());
-    Path storeArchive =
-        HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(), parent,
-            tabledir, td.getColumnFamilies()[0].getName());
+    Path tabledir = FSUtils.getTableDir(rootdir, htd.getTableName());
+    Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
+    Path storeArchive = HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
+      tabledir, htd.getColumnFamilies()[0].getName());
     LOG.debug("Table dir:" + tabledir);
     LOG.debug("Store dir:" + storedir);
     LOG.debug("Store archive dir:" + storeArchive);
 
     // add a couple of store files that we can check for
-    FileStatus[] mockFiles = addMockStoreFiles(2, this.masterServices, storedir);
+    FileStatus[] mockFiles = addMockStoreFiles(2, services, storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
     int index = 0;
@@ -454,10 +613,6 @@ public class TestCatalogJanitor {
 
     // do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, parentMetaRow));
-    Path parentDir = new Path(tabledir, parent.getEncodedName());
-    // Cleanup procedure runs async. Wait till it done.
-    ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
-    assertTrue(!fs.exists(parentDir));
     LOG.debug("Finished cleanup of parent region");
 
     // and now check to make sure that the files have actually been archived
@@ -469,6 +624,8 @@ public class TestCatalogJanitor {
 
     // cleanup
     FSUtils.delete(fs, rootdir, true);
+    services.stop("Test finished");
+    janitor.cancel(true);
   }
 
   /**
@@ -488,62 +645,69 @@ public class TestCatalogJanitor {
    */
   @Test
   public void testDuplicateHFileResolution() throws Exception {
-   TableDescriptor td = createTableDescriptorForCurrentMethod();
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    setRootDirAndCleanIt(htu, "testCleanParent");
+    MasterServices services = new MockMasterServices(htu);
+
+    // create the janitor
+
+    CatalogJanitor janitor = new CatalogJanitor(services);
 
     // Create regions.
-    HRegionInfo parent = new HRegionInfo(td.getTableName(),
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+    htd.addFamily(new HColumnDescriptor("f"));
+    HRegionInfo parent = new HRegionInfo(htd.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
-    HRegionInfo splita = new HRegionInfo(td.getTableName(),
+    HRegionInfo splita = new HRegionInfo(htd.getTableName(),
         Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
-    HRegionInfo splitb = new HRegionInfo(td.getTableName(),
+    HRegionInfo splitb = new HRegionInfo(htd.getTableName(),
         Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
     // Test that when both daughter regions are in place, that we do not
     // remove the parent.
     Result r = createResult(parent, splita, splitb);
-    FileSystem fs = FileSystem.get(HTU.getConfiguration());
-    Path rootdir = this.masterServices.getMasterFileSystem().getRootDir();
-    // Have to set the root directory since we use it in HFileDisposer to figure out to get to the
+
+    FileSystem fs = FileSystem.get(htu.getConfiguration());
+
+    Path rootdir = services.getMasterFileSystem().getRootDir();
+    // have to set the root directory since we use it in HFileDisposer to figure out to get to the
     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
     // the single test passes, but when the full suite is run, things get borked).
     FSUtils.setRootDir(fs.getConf(), rootdir);
     Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
-    Path storedir = HStore.getStoreHomedir(tabledir, parent, td.getColumnFamilies()[0].getName());
+    Path storedir = HStore.getStoreHomedir(tabledir, parent, htd.getColumnFamilies()[0].getName());
     System.out.println("Old root:" + rootdir);
     System.out.println("Old table:" + tabledir);
     System.out.println("Old store:" + storedir);
 
-    Path storeArchive =
-        HFileArchiveUtil.getStoreArchivePath(this.masterServices.getConfiguration(), parent,
-      tabledir, td.getColumnFamilies()[0].getName());
+    Path storeArchive = HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
+      tabledir, htd.getColumnFamilies()[0].getName());
     System.out.println("Old archive:" + storeArchive);
 
     // enable archiving, make sure that files get archived
-    addMockStoreFiles(2, this.masterServices, storedir);
+    addMockStoreFiles(2, services, storedir);
     // get the current store files for comparison
     FileStatus[] storeFiles = fs.listStatus(storedir);
-    // Do the cleaning of the parent
+    // do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, r));
-    Path parentDir = new Path(tabledir, parent.getEncodedName());
-    ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
-    assertTrue(!fs.exists(parentDir));
 
-    // And now check to make sure that the files have actually been archived
+    // and now check to make sure that the files have actually been archived
     FileStatus[] archivedStoreFiles = fs.listStatus(storeArchive);
     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs);
 
     // now add store files with the same names as before to check backup
     // enable archiving, make sure that files get archived
-    addMockStoreFiles(2, this.masterServices, storedir);
+    addMockStoreFiles(2, services, storedir);
 
-    // Do the cleaning of the parent
+    // do the cleaning of the parent
     assertTrue(janitor.cleanParent(parent, r));
-    // Cleanup procedure runs async. Wait till it done.
-    ProcedureTestingUtility.waitAllProcedures(masterServices.getMasterProcedureExecutor());
-    assertTrue(!fs.exists(parentDir));
 
     // and now check to make sure that the files have actually been archived
     archivedStoreFiles = fs.listStatus(storeArchive);
     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs, true);
+
+    // cleanup
+    services.stop("Test finished");
+    janitor.cancel(true);
   }
 
   private FileStatus[] addMockStoreFiles(int count, MasterServices services, Path storedir)
@@ -565,7 +729,8 @@ public class TestCatalogJanitor {
     return storeFiles;
   }
 
-  private String setRootDirAndCleanIt(final HBaseTestingUtility htu, final String subdir)
+  private String setRootDirAndCleanIt(final HBaseTestingUtility htu,
+      final String subdir)
   throws IOException {
     Path testdir = htu.getDataTestDir(subdir);
     FileSystem fs = FileSystem.get(htu.getConfiguration());
@@ -574,14 +739,24 @@ public class TestCatalogJanitor {
     return FSUtils.getRootDir(htu.getConfiguration()).toString();
   }
 
+  /**
+   * @param services Master services instance.
+   * @param htd
+   * @param parent
+   * @param daughter
+   * @param midkey
+   * @param top True if we are to write a 'top' reference.
+   * @return Path to reference we created.
+   * @throws IOException
+   */
   private Path createReferences(final MasterServices services,
-      final TableDescriptor td, final HRegionInfo parent,
+      final HTableDescriptor htd, final HRegionInfo parent,
       final HRegionInfo daughter, final byte [] midkey, final boolean top)
   throws IOException {
     Path rootdir = services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, parent.getTable());
     Path storedir = HStore.getStoreHomedir(tabledir, daughter,
-      td.getColumnFamilies()[0].getName());
+      htd.getColumnFamilies()[0].getName());
     Reference ref =
       top? Reference.createTopReference(midkey): Reference.createBottomReference(midkey);
     long now = System.currentTimeMillis();
@@ -597,4 +772,30 @@ public class TestCatalogJanitor {
   throws IOException {
     return MetaMockingUtil.getMetaTableRowResult(parent, null, a, b);
   }
-}
\ No newline at end of file
+
+  private HTableDescriptor createHTableDescriptor() {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("t"));
+    htd.addFamily(new HColumnDescriptor("f"));
+    return htd;
+  }
+
+  private MultiResponse buildMultiResponse(MultiRequest req) {
+    MultiResponse.Builder builder = MultiResponse.newBuilder();
+    RegionActionResult.Builder regionActionResultBuilder =
+        RegionActionResult.newBuilder();
+    ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
+    for (RegionAction regionAction: req.getRegionActionList()) {
+      regionActionResultBuilder.clear();
+      for (ClientProtos.Action action: regionAction.getActionList()) {
+        roeBuilder.clear();
+        roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
+        roeBuilder.setIndex(action.getIndex());
+        regionActionResultBuilder.addResultOrException(roeBuilder.build());
+      }
+      builder.addRegionActionResult(regionActionResultBuilder.build());
+    }
+    return builder.build();
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 fcd2cf9..b78bfd1 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,7 +84,6 @@ 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/a3c5a744/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 d6210b9..eb4ce99 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,7 +42,6 @@ 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/a3c5a744/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 0073cdf..74f2c91 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
@@ -34,11 +34,9 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-@Ignore // SimpleLoadBalancer seems borked whether AMv2 or not. Disabling till gets attention.
 @Category({MasterTests.class, MediumTests.class})
 public class TestMasterBalanceThrottling {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -122,9 +120,8 @@ public class TestMasterBalanceThrottling {
       @Override
       public void run() {
         while (!stop.get()) {
-          maxCount.set(Math.max(maxCount.get(),
-              master.getAssignmentManager().getRegionStates()
-              .getRegionsInTransition().size()));
+          maxCount.set(Math.max(maxCount.get(), master.getAssignmentManager().getRegionStates()
+              .getRegionsInTransitionCount()));
           try {
             Thread.sleep(10);
           } catch (InterruptedException e) {
@@ -139,7 +136,7 @@ public class TestMasterBalanceThrottling {
   }
 
   private void unbalance(HMaster master, TableName tableName) throws Exception {
-    while (master.getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0) {
+    while (master.getAssignmentManager().getRegionStates().getRegionsInTransitionCount() > 0) {
       Thread.sleep(100);
     }
     HRegionServer biasedServer = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
@@ -147,7 +144,7 @@ public class TestMasterBalanceThrottling {
       master.move(regionInfo.getEncodedNameAsBytes(),
         Bytes.toBytes(biasedServer.getServerName().getServerName()));
     }
-    while (master.getAssignmentManager().getRegionStates().getRegionsInTransition().size() > 0) {
+    while (master.getAssignmentManager().getRegionStates().getRegionsInTransitionCount() > 0) {
       Thread.sleep(100);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 29c0576..f57d6b9 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,8 +43,6 @@ 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;
@@ -57,12 +55,10 @@ import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({FlakeyTests.class, LargeTests.class})
-@Ignore // Needs to be rewritten for AMv2. Uses tricks not ordained when up on AMv2.
 public class TestMasterFailover {
   private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
 
@@ -256,22 +252,23 @@ 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.CLOSING, oldState.getServerName());
-    stateStore.updateRegionState(HConstants.NO_SEQNUM, -1, newState, oldState);
+    RegionState newState = new RegionState(
+      hriOnline, State.PENDING_CLOSE, 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.OPENING, newState.getServerName());
-    stateStore.updateRegionState(HConstants.NO_SEQNUM, -1, newState, oldState);
+    newState = new RegionState(hriOffline, State.PENDING_OPEN, 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.CLOSING);
+    oldState = new RegionState(failedClose, State.PENDING_CLOSE);
     newState = new RegionState(failedClose, State.FAILED_CLOSE, newState.getServerName());
-    stateStore.updateRegionState(HConstants.NO_SEQNUM, -1, newState, oldState);
+    stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
     HRegionInfo failedOpen = new HRegionInfo(offlineTable.getTableName(), null, null);
     createRegion(failedOpen, rootdir, conf, offlineTable);
@@ -279,9 +276,9 @@ 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.OPENING);
+    oldState = new RegionState(failedOpen, State.PENDING_OPEN);
     newState = new RegionState(failedOpen, State.FAILED_OPEN, newState.getServerName());
-    stateStore.updateRegionState(HConstants.NO_SEQNUM, -1, newState, oldState);
+    stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
     HRegionInfo failedOpenNullServer = new HRegionInfo(offlineTable.getTableName(), null, null);
     LOG.info("Failed open NUll server " + failedOpenNullServer.getEncodedName());
@@ -292,7 +289,7 @@ public class TestMasterFailover {
     // the region
     oldState = new RegionState(failedOpenNullServer, State.OFFLINE);
     newState = new RegionState(failedOpenNullServer, State.FAILED_OPEN, null);
-    stateStore.updateRegionState(HConstants.NO_SEQNUM, -1, newState, oldState);
+    stateStore.updateRegionState(HConstants.NO_SEQNUM, newState, oldState);
 
     // Stop the master
     log("Aborting master");
@@ -381,12 +378,12 @@ public class TestMasterFailover {
     assertEquals("hbase:meta should be onlined on RS",
       metaState.getState(), State.OPEN);
 
-    // Update meta state as OPENING, then kill master
+    // Update meta state as PENDING_OPEN, 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.OPENING);
+      rs.getServerName(), State.PENDING_OPEN);
     Region meta = rs.getFromOnlineRegions(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     rs.removeFromOnlineRegions(meta, null);
     ((HRegion)meta).close();
@@ -413,12 +410,12 @@ public class TestMasterFailover {
     assertEquals("hbase:meta should be onlined on RS",
       metaState.getState(), State.OPEN);
 
-    // Update meta state as CLOSING, then kill master
+    // Update meta state as PENDING_CLOSE, 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.CLOSING);
+      rs.getServerName(), State.PENDING_CLOSE);
 
     log("Aborting master");
     activeMaster.abort("test-kill");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
index 8a216c5..0084d44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -56,13 +55,12 @@ public class TestMasterMetrics {
         KeeperException, InterruptedException {
       super(conf, cp);
     }
-/*
+
     @Override
     protected void tryRegionServerReport(
         long reportStartTime, long reportEndTime) {
       // do nothing
     }
-*/
   }
 
   @BeforeClass
@@ -83,7 +81,7 @@ public class TestMasterMetrics {
     }
   }
 
-  @Ignore @Test(timeout = 300000)
+  @Test(timeout = 300000)
   public void testClusterRequests() throws Exception {
 
     // sending fake request to master to see how metric value has changed
@@ -116,7 +114,7 @@ public class TestMasterMetrics {
     master.stopMaster();
   }
 
-  @Ignore @Test
+  @Test
   public void testDefaultMasterMetrics() throws Exception {
     MetricsMasterSource masterSource = master.getMasterMetrics().getMetricsSource();
     metricsHelper.assertGauge( "numRegionServers", 2, masterSource);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 8b0874a..6c737e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -173,7 +172,7 @@ public class TestMasterOperationsForRegionReplicas {
       }
       validateFromSnapshotFromMeta(TEST_UTIL, tableName, numRegions, numReplica,
         ADMIN.getConnection());
-      /* DISABLED!!!!! FOR NOW!!!!
+
       // Now shut the whole cluster down, and verify the assignments are kept so that the
       // availability constraints are met.
       TEST_UTIL.getConfiguration().setBoolean("hbase.master.startup.retainassign", true);
@@ -193,19 +192,17 @@ public class TestMasterOperationsForRegionReplicas {
         TEST_UTIL.getMiniHBaseCluster().startRegionServer();
       }
 
-      // Check on alter table
+      //check on alter table
       ADMIN.disableTable(tableName);
       assert(ADMIN.isTableDisabled(tableName));
       //increase the replica
       desc.setRegionReplication(numReplica + 1);
       ADMIN.modifyTable(tableName, desc);
       ADMIN.enableTable(tableName);
-      LOG.info(ADMIN.getTableDescriptor(tableName).toString());
       assert(ADMIN.isTableEnabled(tableName));
       List<HRegionInfo> regions = TEST_UTIL.getMiniHBaseCluster().getMaster()
           .getAssignmentManager().getRegionStates().getRegionsOfTable(tableName);
-      assertTrue("regions.size=" + regions.size() + ", numRegions=" + numRegions + ", numReplica=" + numReplica,
-          regions.size() == numRegions * (numReplica + 1));
+      assert(regions.size() == numRegions * (numReplica + 1));
 
       //decrease the replica(earlier, table was modified to have a replica count of numReplica + 1)
       ADMIN.disableTable(tableName);
@@ -232,7 +229,6 @@ public class TestMasterOperationsForRegionReplicas {
       assert(defaultReplicas.size() == numRegions);
       Collection<Integer> counts = new HashSet<>(defaultReplicas.values());
       assert(counts.size() == 1 && counts.contains(new Integer(numReplica)));
-      */
     } finally {
       ADMIN.disableTable(tableName);
       ADMIN.deleteTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 23efdb2..b59e6ff 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
@@ -18,12 +18,15 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+
 import java.io.IOException;
 import java.io.StringWriter;
-import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.TreeSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -32,23 +35,23 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
-import org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapperStub;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServer;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapperStub;
+import org.apache.hadoop.hbase.tmpl.master.AssignmentManagerStatusTmpl;
+import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /**
  * Tests for the master status page and its template.
@@ -87,7 +90,7 @@ public class TestMasterStatusServlet {
     // Fake AssignmentManager and RIT
     AssignmentManager am = Mockito.mock(AssignmentManager.class);
     RegionStates rs = Mockito.mock(RegionStates.class);
-    List<RegionState> regionsInTransition = new ArrayList<>();
+    Set<RegionState> regionsInTransition = new HashSet<>();
     regionsInTransition.add(new RegionState(FAKE_HRI, RegionState.State.CLOSING, 12345L, FAKE_HOST));
     Mockito.doReturn(rs).when(am).getRegionStates();
     Mockito.doReturn(regionsInTransition).when(rs).getRegionsInTransition();
@@ -154,4 +157,45 @@ public class TestMasterStatusServlet {
       .setDeadServers(deadServers)
       .render(new StringWriter(), master);
   }
+
+  @Test
+  public void testAssignmentManagerTruncatedList() throws IOException {
+    AssignmentManager am = Mockito.mock(AssignmentManager.class);
+    RegionStates rs = Mockito.mock(RegionStates.class);
+
+    // Add 100 regions as in-transition
+    TreeSet<RegionState> regionsInTransition = new TreeSet<>(RegionStates.REGION_STATE_COMPARATOR);
+    for (byte i = 0; i < 100; i++) {
+      HRegionInfo hri = new HRegionInfo(FAKE_TABLE.getTableName(),
+          new byte[]{i}, new byte[]{(byte) (i+1)});
+      regionsInTransition.add(
+        new RegionState(hri, RegionState.State.CLOSING, 12345L, FAKE_HOST));
+    }
+    // Add hbase:meta in transition as well
+    regionsInTransition.add(
+        new RegionState(HRegionInfo.FIRST_META_REGIONINFO,
+                        RegionState.State.CLOSING, 123L, FAKE_HOST));
+    Mockito.doReturn(rs).when(am).getRegionStates();
+    Mockito.doReturn(regionsInTransition).when(rs).getRegionsInTransition();
+    Mockito.doReturn(regionsInTransition).when(rs).getRegionsInTransitionOrderedByTimestamp();
+
+    // Render to a string
+    StringWriter sw = new StringWriter();
+    new AssignmentManagerStatusTmpl()
+      // NOT IMPLEMENTED!!!! .setLimit(50)
+      .render(sw, am);
+    String result = sw.toString();
+    // Should always include META
+    assertTrue(result.contains(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()));
+
+    /* BROKEN BY  HBASE-13839 Fix AssgnmentManagerTmpl.jamon issues (coloring, content etc.) FIX!!
+    // Make sure we only see 50 of them
+    Matcher matcher = Pattern.compile("CLOSING").matcher(result);
+    int count = 0;
+    while (matcher.find()) {
+      count++;
+    }
+    assertEquals(50, count);
+    */
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java
index 8641b20..782c400 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterWalManager.java
@@ -81,7 +81,7 @@ public class TestMasterWalManager {
     // Create a ZKW to use in the test
     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
     zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, walPath),
-      new SplitLogTask.Owned(inRecoveryServerName).toByteArray(),
+      new SplitLogTask.Owned(inRecoveryServerName, mwm.getLogRecoveryMode()).toByteArray(),
         Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
     String staleRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, staleRegion);
     ZKUtil.createWithParents(zkw, staleRegionPath);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 68160df..a845a73 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,7 +30,6 @@ 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/a3c5a744/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 fe5883b..daf6d43 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,19 +35,14 @@ public class TestRegionState {
   public TestName name = new TestName();
 
   @Test
-  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);
+  public void test() {
+    RegionState state1 = new RegionState(
+            new HRegionInfo(TableName.valueOf(name.getMethodName())), RegionState.State.OPENING);
     ClusterStatusProtos.RegionState protobuf1 = state1.convert();
     RegionState state2 = RegionState.convert(protobuf1);
     ClusterStatusProtos.RegionState protobuf2 = state1.convert();
-    assertEquals("RegionState does not match " + state, state1, state2);
-    assertEquals("Protobuf does not match " + state, protobuf1, protobuf2);
+
+    assertEquals(state1, state2);
+    assertEquals(protobuf1, protobuf2);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..17004ec
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionStates.java
@@ -0,0 +1,144 @@
+/**
+ * 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/a3c5a744/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 351fca4..7c41c0f 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,14 +35,12 @@ 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;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -109,7 +107,6 @@ public class TestRestartCluster {
    * This tests retaining assignments on a cluster restart
    */
   @Test (timeout=300000)
-  @Ignore // Does not work in new AMv2 currently.
   public void testRetainAssignmentOnRestart() throws Exception {
     UTIL.startMiniCluster(2);
     while (!UTIL.getMiniHBaseCluster().getMaster().isInitialized()) {
@@ -198,7 +195,7 @@ public class TestRestartCluster {
       Threads.sleep(100);
     }
 
-    snapshot = new SnapshotOfRegionAssignmentFromMeta(master.getConnection());
+    snapshot =new SnapshotOfRegionAssignmentFromMeta(master.getConnection());
     snapshot.initialize();
     Map<HRegionInfo, ServerName> newRegionToRegionServerMap =
       snapshot.getRegionToRegionServerMap();
@@ -207,8 +204,7 @@ public class TestRestartCluster {
       if (TableName.NAMESPACE_TABLE_NAME.equals(entry.getKey().getTable())) continue;
       ServerName oldServer = regionToRegionServerMap.get(entry.getKey());
       ServerName currentServer = entry.getValue();
-      LOG.info("Key=" + entry.getKey() + " oldServer=" + oldServer + ", currentServer=" + currentServer);
-      assertEquals(entry.getKey().toString(), oldServer.getAddress(), currentServer.getAddress());
+      assertEquals(oldServer.getHostAndPort(), currentServer.getHostAndPort());
       assertNotEquals(oldServer.getStartcode(), currentServer.getStartcode());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
index 58be83b..ec7ffe6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestWarmupRegion.java
@@ -19,10 +19,7 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.apache.hadoop.hbase.regionserver.HRegion.warmupHRegion;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,20 +29,21 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 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.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 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.Bytes;
-import org.junit.After;
+import org.junit.experimental.categories.Category;
+import org.junit.BeforeClass;
 import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.After;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 /**
  * Run tests that use the HBase clients; {@link org.apache.hadoop.hbase.client.HTable}.
@@ -160,8 +158,6 @@ public class TestWarmupRegion {
      for (int i = 0; i < 10; i++) {
        HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(serverid);
        byte [] destName = Bytes.toBytes(rs.getServerName().toString());
-       assertTrue(destName != null);
-       LOG.info("i=" + i );
        TEST_UTIL.getMiniHBaseCluster().getMaster().move(info.getEncodedNameAsBytes(), destName);
        serverid = (serverid + 1) % 2;
      }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 07b989b..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
+++ /dev/null
@@ -1,125 +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.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


[24/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 812cf3b..711b9c8 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
@@ -728,40 +728,6 @@ public final class AdminProtos {
      * <code>optional bool isRecovering = 3;</code>
      */
     boolean getIsRecovering();
-
-    /**
-     * <pre>
-     * True if region is splittable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool splittable = 4;</code>
-     */
-    boolean hasSplittable();
-    /**
-     * <pre>
-     * True if region is splittable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool splittable = 4;</code>
-     */
-    boolean getSplittable();
-
-    /**
-     * <pre>
-     * True if region is mergeable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool mergeable = 5;</code>
-     */
-    boolean hasMergeable();
-    /**
-     * <pre>
-     * True if region is mergeable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool mergeable = 5;</code>
-     */
-    boolean getMergeable();
   }
   /**
    * Protobuf type {@code hbase.pb.GetRegionInfoResponse}
@@ -777,8 +743,6 @@ public final class AdminProtos {
     private GetRegionInfoResponse() {
       compactionState_ = 0;
       isRecovering_ = false;
-      splittable_ = false;
-      mergeable_ = false;
     }
 
     @java.lang.Override
@@ -838,16 +802,6 @@ public final class AdminProtos {
               isRecovering_ = input.readBool();
               break;
             }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              splittable_ = input.readBool();
-              break;
-            }
-            case 40: {
-              bitField0_ |= 0x00000010;
-              mergeable_ = input.readBool();
-              break;
-            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -1033,52 +987,6 @@ public final class AdminProtos {
       return isRecovering_;
     }
 
-    public static final int SPLITTABLE_FIELD_NUMBER = 4;
-    private boolean splittable_;
-    /**
-     * <pre>
-     * True if region is splittable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool splittable = 4;</code>
-     */
-    public boolean hasSplittable() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <pre>
-     * True if region is splittable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool splittable = 4;</code>
-     */
-    public boolean getSplittable() {
-      return splittable_;
-    }
-
-    public static final int MERGEABLE_FIELD_NUMBER = 5;
-    private boolean mergeable_;
-    /**
-     * <pre>
-     * True if region is mergeable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool mergeable = 5;</code>
-     */
-    public boolean hasMergeable() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <pre>
-     * True if region is mergeable, false otherwise.
-     * </pre>
-     *
-     * <code>optional bool mergeable = 5;</code>
-     */
-    public boolean getMergeable() {
-      return mergeable_;
-    }
-
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -1108,12 +1016,6 @@ public final class AdminProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeBool(3, isRecovering_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBool(4, splittable_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeBool(5, mergeable_);
-      }
       unknownFields.writeTo(output);
     }
 
@@ -1134,14 +1036,6 @@ public final class AdminProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, isRecovering_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeBoolSize(4, splittable_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeBoolSize(5, mergeable_);
-      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -1173,16 +1067,6 @@ public final class AdminProtos {
         result = result && (getIsRecovering()
             == other.getIsRecovering());
       }
-      result = result && (hasSplittable() == other.hasSplittable());
-      if (hasSplittable()) {
-        result = result && (getSplittable()
-            == other.getSplittable());
-      }
-      result = result && (hasMergeable() == other.hasMergeable());
-      if (hasMergeable()) {
-        result = result && (getMergeable()
-            == other.getMergeable());
-      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -1207,16 +1091,6 @@ public final class AdminProtos {
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsRecovering());
       }
-      if (hasSplittable()) {
-        hash = (37 * hash) + SPLITTABLE_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
-            getSplittable());
-      }
-      if (hasMergeable()) {
-        hash = (37 * hash) + MERGEABLE_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
-            getMergeable());
-      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1346,10 +1220,6 @@ public final class AdminProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         isRecovering_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
-        splittable_ = false;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        mergeable_ = false;
-        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -1390,14 +1260,6 @@ public final class AdminProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.isRecovering_ = isRecovering_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.splittable_ = splittable_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.mergeable_ = mergeable_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1449,12 +1311,6 @@ public final class AdminProtos {
         if (other.hasIsRecovering()) {
           setIsRecovering(other.getIsRecovering());
         }
-        if (other.hasSplittable()) {
-          setSplittable(other.getSplittable());
-        }
-        if (other.hasMergeable()) {
-          setMergeable(other.getMergeable());
-        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -1674,102 +1530,6 @@ public final class AdminProtos {
         onChanged();
         return this;
       }
-
-      private boolean splittable_ ;
-      /**
-       * <pre>
-       * True if region is splittable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool splittable = 4;</code>
-       */
-      public boolean hasSplittable() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <pre>
-       * True if region is splittable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool splittable = 4;</code>
-       */
-      public boolean getSplittable() {
-        return splittable_;
-      }
-      /**
-       * <pre>
-       * True if region is splittable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool splittable = 4;</code>
-       */
-      public Builder setSplittable(boolean value) {
-        bitField0_ |= 0x00000008;
-        splittable_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * True if region is splittable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool splittable = 4;</code>
-       */
-      public Builder clearSplittable() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        splittable_ = false;
-        onChanged();
-        return this;
-      }
-
-      private boolean mergeable_ ;
-      /**
-       * <pre>
-       * True if region is mergeable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool mergeable = 5;</code>
-       */
-      public boolean hasMergeable() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <pre>
-       * True if region is mergeable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool mergeable = 5;</code>
-       */
-      public boolean getMergeable() {
-        return mergeable_;
-      }
-      /**
-       * <pre>
-       * True if region is mergeable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool mergeable = 5;</code>
-       */
-      public Builder setMergeable(boolean value) {
-        bitField0_ |= 0x00000010;
-        mergeable_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * True if region is mergeable, false otherwise.
-       * </pre>
-       *
-       * <code>optional bool mergeable = 5;</code>
-       */
-      public Builder clearMergeable() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        mergeable_ = false;
-        onChanged();
-        return this;
-      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -9654,70 +9414,53 @@ public final class AdminProtos {
 
   }
 
-  public interface FlushRegionRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionRequest)
+  public interface CloseRegionForSplitOrMergeRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
-     */
-    boolean hasRegion();
-    /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion();
-    /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder();
-
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> 
+        getRegionList();
     /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    boolean hasIfOlderThanTs();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
     /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    long getIfOlderThanTs();
-
+    int getRegionCount();
     /**
-     * <pre>
-     * whether to write a marker to WAL even if not flushed
-     * </pre>
-     *
-     * <code>optional bool write_flush_wal_marker = 3;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    boolean hasWriteFlushWalMarker();
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
+        getRegionOrBuilderList();
     /**
-     * <pre>
-     * whether to write a marker to WAL even if not flushed
-     * </pre>
-     *
-     * <code>optional bool write_flush_wal_marker = 3;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    boolean getWriteFlushWalMarker();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+        int index);
   }
   /**
    * <pre>
    **
-   * Flushes the MemStore of the specified region.
-   * &lt;p&gt;
-   * This method is synchronous.
+   * Closes the specified region(s) for
+   * split or merge
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.FlushRegionRequest}
+   * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
    */
-  public  static final class FlushRegionRequest extends
+  public  static final class CloseRegionForSplitOrMergeRequest 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.CloseRegionForSplitOrMergeRequest)
+      CloseRegionForSplitOrMergeRequestOrBuilder {
+    // Use CloseRegionForSplitOrMergeRequest.newBuilder() to construct.
+    private CloseRegionForSplitOrMergeRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
       super(builder);
     }
-    private FlushRegionRequest() {
-      ifOlderThanTs_ = 0L;
-      writeFlushWalMarker_ = false;
+    private CloseRegionForSplitOrMergeRequest() {
+      region_ = java.util.Collections.emptyList();
     }
 
     @java.lang.Override
@@ -9725,7 +9468,7 @@ public final class AdminProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private FlushRegionRequest(
+    private CloseRegionForSplitOrMergeRequest(
         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 {
@@ -9749,26 +9492,12 @@ public final class AdminProtos {
               break;
             }
             case 10: {
-              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();
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
+                mutable_bitField0_ |= 0x00000001;
               }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 16: {
-              bitField0_ |= 0x00000002;
-              ifOlderThanTs_ = input.readUInt64();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              writeFlushWalMarker_ = input.readBool();
+              region_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
               break;
             }
           }
@@ -9779,80 +9508,58 @@ 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_FlushRegionRequest_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_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_FlushRegionRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
     }
 
-    private int bitField0_;
     public static final int REGION_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
     /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public boolean hasRegion() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+      return region_;
     }
     /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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_;
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> 
+        getRegionOrBuilderList() {
+      return region_;
     }
     /**
-     * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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 int getRegionCount() {
+      return region_.size();
     }
-
-    public static final int IF_OLDER_THAN_TS_FIELD_NUMBER = 2;
-    private long ifOlderThanTs_;
     /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public boolean hasIfOlderThanTs() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+      return region_.get(index);
     }
     /**
-     * <code>optional uint64 if_older_than_ts = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+        int index) {
+      return region_.get(index);
     }
 
     private byte memoizedIsInitialized = -1;
@@ -9861,13 +9568,11 @@ public final class AdminProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      if (!hasRegion()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      if (!getRegion().isInitialized()) {
-        memoizedIsInitialized = 0;
-        return false;
+      for (int i = 0; i < getRegionCount(); i++) {
+        if (!getRegion(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
       }
       memoizedIsInitialized = 1;
       return true;
@@ -9875,14 +9580,8 @@ 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.writeMessage(1, getRegion());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt64(2, ifOlderThanTs_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBool(3, writeFlushWalMarker_);
+      for (int i = 0; i < region_.size(); i++) {
+        output.writeMessage(1, region_.get(i));
       }
       unknownFields.writeTo(output);
     }
@@ -9892,17 +9591,9 @@ public final class AdminProtos {
       if (size != -1) return size;
 
       size = 0;
-      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)) {
+      for (int i = 0; i < region_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeBoolSize(3, writeFlushWalMarker_);
+          .computeMessageSize(1, region_.get(i));
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -9915,27 +9606,14 @@ public final class AdminProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) obj;
 
       boolean result = true;
-      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 && getRegionList()
+          .equals(other.getRegionList());
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -9947,77 +9625,67 @@ public final class AdminProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().hashCode();
-      if (hasRegion()) {
+      if (getRegionCount() > 0) {
         hash = (37 * hash) + REGION_FIELD_NUMBER;
-        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 = (53 * hash) + getRegionList().hashCode();
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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 {
@@ -10029,7 +9697,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.FlushRegionRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -10046,30 +9714,29 @@ public final class AdminProtos {
     /**
      * <pre>
      **
-     * Flushes the MemStore of the specified region.
-     * &lt;p&gt;
-     * This method is synchronous.
+     * Closes the specified region(s) for
+     * split or merge
      * </pre>
      *
-     * Protobuf type {@code hbase.pb.FlushRegionRequest}
+     * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeRequest}
      */
     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.FlushRegionRequest)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequestOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequestOrBuilder {
       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_CloseRegionForSplitOrMergeRequest_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_FlushRegionRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -10088,56 +9755,43 @@ public final class AdminProtos {
       public Builder clear() {
         super.clear();
         if (regionBuilder_ == null) {
-          region_ = null;
+          region_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
         } 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_FlushRegionRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeRequest_descriptor;
       }
 
-      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 getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      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);
+      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);
         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;
       }
@@ -10169,24 +9823,41 @@ 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.FlushRegionRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest)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);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      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());
+      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_);
+            }
+          }
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -10194,11 +9865,10 @@ public final class AdminProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasRegion()) {
-          return false;
-        }
-        if (!getRegion().isInitialized()) {
-          return false;
+        for (int i = 0; i < getRegionCount(); i++) {
+          if (!getRegion(i).isInitialized()) {
+            return false;
+          }
         }
         return true;
       }
@@ -10207,11 +9877,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.FlushRegionRequest parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest 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.FlushRegionRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -10222,202 +9892,244 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier region_ = null;
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+      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>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public boolean hasRegion() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+      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>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() {
+      public int getRegionCount() {
         if (regionBuilder_ == null) {
-          return region_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+          return region_.size();
         } else {
-          return regionBuilder_.getMessage();
+          return regionBuilder_.getCount();
         }
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+        if (regionBuilder_ == null) {
+          return region_.get(index);
+        } else {
+          return regionBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
+       */
+      public Builder setRegion(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
         if (regionBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          region_ = value;
+          ensureRegionIsMutable();
+          region_.set(index, value);
           onChanged();
         } else {
-          regionBuilder_.setMessage(value);
+          regionBuilder_.setMessage(index, value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
       public Builder setRegion(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
         if (regionBuilder_ == null) {
-          region_ = builderForValue.build();
+          ensureRegionIsMutable();
+          region_.set(index, builderForValue.build());
           onChanged();
         } else {
-          regionBuilder_.setMessage(builderForValue.build());
+          regionBuilder_.setMessage(index, builderForValue.build());
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder mergeRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+      public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
         if (regionBuilder_ == null) {
-          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;
+          if (value == null) {
+            throw new NullPointerException();
           }
+          ensureRegionIsMutable();
+          region_.add(value);
           onChanged();
         } else {
-          regionBuilder_.mergeFrom(value);
+          regionBuilder_.addMessage(value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder clearRegion() {
+      public Builder addRegion(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
         if (regionBuilder_ == null) {
-          region_ = null;
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionIsMutable();
+          region_.add(index, value);
           onChanged();
         } else {
-          regionBuilder_.clear();
+          regionBuilder_.addMessage(index, value);
         }
-        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getRegionFieldBuilder().getBuilder();
+      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;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() {
-        if (regionBuilder_ != null) {
-          return regionBuilder_.getMessageOrBuilder();
+      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 {
-          return region_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : region_;
+          regionBuilder_.addMessage(index, builderForValue.build());
         }
+        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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() {
+      public Builder addAllRegion(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> values) {
         if (regionBuilder_ == null) {
-          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;
+          ensureRegionIsMutable();
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+              values, region_);
+          onChanged();
+        } else {
+          regionBuilder_.addAllMessages(values);
         }
-        return regionBuilder_;
+        return this;
       }
-
-      private long ifOlderThanTs_ ;
       /**
-       * <code>optional uint64 if_older_than_ts = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public boolean hasIfOlderThanTs() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+      public Builder clearRegion() {
+        if (regionBuilder_ == null) {
+          region_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
+          onChanged();
+        } else {
+          regionBuilder_.clear();
+        }
+        return this;
       }
       /**
-       * <code>optional uint64 if_older_than_ts = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public long getIfOlderThanTs() {
-        return ifOlderThanTs_;
-      }
+      public Builder removeRegion(int index) {
+        if (regionBuilder_ == null) {
+          ensureRegionIsMutable();
+          region_.remove(index);
+          onChanged();
+        } else {
+          regionBuilder_.remove(index);
+        }
+        return this;
+      }
       /**
-       * <code>optional uint64 if_older_than_ts = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setIfOlderThanTs(long value) {
-        bitField0_ |= 0x00000002;
-        ifOlderThanTs_ = value;
-        onChanged();
-        return this;
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder(
+          int index) {
+        return getRegionFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>optional uint64 if_older_than_ts = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder clearIfOlderThanTs() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        ifOlderThanTs_ = 0L;
-        onChanged();
-        return this;
+      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);
+        }
       }
-
-      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>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public boolean hasWriteFlushWalMarker() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+      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_);
+        }
       }
       /**
-       * <pre>
-       * whether to write a marker to WAL even if not flushed
-       * </pre>
-       *
-       * <code>optional bool write_flush_wal_marker = 3;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public boolean getWriteFlushWalMarker() {
-        return writeFlushWalMarker_;
+      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());
       }
       /**
-       * <pre>
-       * whether to write a marker to WAL even if not flushed
-       * </pre>
-       *
-       * <code>optional bool write_flush_wal_marker = 3;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setWriteFlushWalMarker(boolean value) {
-        bitField0_ |= 0x00000004;
-        writeFlushWalMarker_ = value;
-        onChanged();
-        return this;
+      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());
       }
       /**
-       * <pre>
-       * whether to write a marker to WAL even if not flushed
-       * </pre>
-       *
-       * <code>optional bool write_flush_wal_marker = 3;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder clearWriteFlushWalMarker() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        writeFlushWalMarker_ = false;
-        onChanged();
-        return this;
+      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 final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
@@ -10430,90 +10142,70 @@ public final class AdminProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.FlushRegionRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.FlushRegionRequest)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest DEFAULT_INSTANCE;
+    // @@protoc_insertion_point(class_scope:hbase.pb.CloseRegionForSplitOrMergeRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest();
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest();
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest getDefaultInstance() {
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @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(
+    @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(
           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 FlushRegionRequest(input, extensionRegistry);
+          return new CloseRegionForSplitOrMergeRequest(input, extensionRegistry);
       }
     };
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionRequest> parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<FlushRegionRequest> getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<CloseRegionForSplitOrMergeRequest> getParserForType() {
       return PARSER;
     }
 
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest getDefaultInstanceForType() {
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeRequest getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface FlushRegionResponseOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.FlushRegionResponse)
+  public interface CloseRegionForSplitOrMergeResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.CloseRegionForSplitOrMergeResponse)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>required uint64 last_flush_time = 1;</code>
-     */
-    boolean hasLastFlushTime();
-    /**
-     * <code>required uint64 last_flush_time = 1;</code>
-     */
-    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>
+     * <code>required bool closed = 1;</code>
      */
-    boolean hasWroteFlushWalMarker();
+    boolean hasClosed();
     /**
-     * <code>optional bool wrote_flush_wal_marker = 3;</code>
+     * <code>required bool closed = 1;</code>
      */
-    boolean getWroteFlushWalMarker();
+    boolean getClosed();
   }
   /**
-   * Protobuf type {@code hbase.pb.FlushRegionResponse}
+   * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
    */
-  public  static final class FlushRegionResponse extends
+  public  static final class CloseRegionForSplitOrMergeResponse extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@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) {
+      // @@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) {
       super(builder);
     }
-    private FlushRegionResponse() {
-      lastFlushTime_ = 0L;
-      flushed_ = false;
-      wroteFlushWalMarker_ = false;
+    private CloseRegionForSplitOrMergeResponse() {
+      closed_ = false;
     }
 
     @java.lang.Override
@@ -10521,7 +10213,7 @@ public final class AdminProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private FlushRegionResponse(
+    private CloseRegionForSplitOrMergeResponse(
         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 {
@@ -10546,17 +10238,7 @@ public final class AdminProtos {
             }
             case 8: {
               bitField0_ |= 0x00000001;
-              lastFlushTime_ = input.readUInt64();
-              break;
-            }
-            case 16: {
-              bitField0_ |= 0x00000002;
-              flushed_ = input.readBool();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              wroteFlushWalMarker_ = input.readBool();
+              closed_ = input.readBool();
               break;
             }
           }
@@ -10573,60 +10255,30 @@ 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_FlushRegionResponse_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_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_FlushRegionResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
     }
 
     private int bitField0_;
-    public static final int LAST_FLUSH_TIME_FIELD_NUMBER = 1;
-    private long lastFlushTime_;
+    public static final int CLOSED_FIELD_NUMBER = 1;
+    private boolean closed_;
     /**
-     * <code>required uint64 last_flush_time = 1;</code>
+     * <code>required bool closed = 1;</code>
      */
-    public boolean hasLastFlushTime() {
+    public boolean hasClosed() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>required uint64 last_flush_time = 1;</code>
-     */
-    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>
+     * <code>required bool closed = 1;</code>
      */
-    public boolean getWroteFlushWalMarker() {
-      return wroteFlushWalMarker_;
+    public boolean getClosed() {
+      return closed_;
     }
 
     private byte memoizedIsInitialized = -1;
@@ -10635,7 +10287,7 @@ public final class AdminProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 0) return false;
 
-      if (!hasLastFlushTime()) {
+      if (!hasClosed()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -10646,13 +10298,7 @@ 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.writeUInt64(1, lastFlushTime_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBool(2, flushed_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeBool(3, wroteFlushWalMarker_);
+        output.writeBool(1, closed_);
       }
       unknownFields.writeTo(output);
     }
@@ -10664,15 +10310,7 @@ public final class AdminProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .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_);
+          .computeBoolSize(1, closed_);
       }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
@@ -10685,26 +10323,16 @@ public final class AdminProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) obj;
 
       boolean result = true;
-      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 && (hasClosed() == other.hasClosed());
+      if (hasClosed()) {
+        result = result && (getClosed()
+            == other.getClosed());
       }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
@@ -10717,78 +10345,68 @@ public final class AdminProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().hashCode();
-      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(
-            getFlushed());
-      }
-      if (hasWroteFlushWalMarker()) {
-        hash = (37 * hash) + WROTE_FLUSH_WAL_MARKER_FIELD_NUMBER;
+      if (hasClosed()) {
+        hash = (37 * hash) + CLOSED_FIELD_NUMBER;
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
-            getWroteFlushWalMarker());
+            getClosed());
       }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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 {
@@ -10800,7 +10418,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.FlushRegionResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -10815,25 +10433,25 @@ public final class AdminProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.FlushRegionResponse}
+     * Protobuf type {@code hbase.pb.CloseRegionForSplitOrMergeResponse}
      */
     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.FlushRegionResponse)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponseOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.CloseRegionForSplitOrMergeResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponseOrBuilder {
       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_FlushRegionResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_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_FlushRegionResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -10850,48 +10468,36 @@ public final class AdminProtos {
       }
       public Builder clear() {
         super.clear();
-        lastFlushTime_ = 0L;
+        closed_ = false;
         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_FlushRegionResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.internal_static_hbase_pb_CloseRegionForSplitOrMergeResponse_descriptor;
       }
 
-      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 getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      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);
+      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);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        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.closed_ = closed_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -10924,24 +10530,18 @@ 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.FlushRegionResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse)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);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      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());
+      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());
         }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
@@ -10949,7 +10549,7 @@ public final class AdminProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasLastFlushTime()) {
+        if (!hasClosed()) {
           return false;
         }
         return true;
@@ -10959,11 +10559,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.FlushRegionResponse parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse 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.FlushRegionResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionForSplitOrMergeResponse) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -10974,98 +10574,34 @@ public final class AdminProtos {
       }
       private int bitField0_;
 
-      private long lastFlushTime_ ;
+      private boolean closed_ ;
       /**
-       * <code>required uint64 last_flush_time = 1;</code>
+       * <code>required bool closed = 1;</code>
        */
-      public boolean hasLastFlushTime() {
+      public boolean hasClosed() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>required uint64 last_flush_time = 1;</code>
+       * <code>required bool closed = 1;</code>
        */
-      public long getLastFlushTime() {
-        return lastFlushTime_;
+      public boolean getClosed() {
+        return closed_;
       }
       /**
-       * <code>required uint64 last_flush_time = 1;</code>
+       * <code>required bool closed = 1;</code>
        */
-      public Builder setLastFlushTime(long value) {
+      public Builder setClosed(boolean value) {
         bitField0_ |= 0x00000001;
-        lastFlushTime_ = value;
+        closed_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>required uint64 last_flush_time = 1;</code>
+       * <code>required bool closed = 1;</code>
        */
-      public Builder clearLastFl

<TRUNCATED>

[23/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 e1c8d97..d7bbd05 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
@@ -1734,201 +1734,100 @@ public final class MasterProcedureProtos {
   }
 
   /**
-   * Protobuf enum {@code hbase.pb.DispatchMergingRegionsState}
+   * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
    */
-  public enum DispatchMergingRegionsState
+  public enum MergeTableRegionsState
       implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
+     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
      */
-    DISPATCH_MERGING_REGIONS_PREPARE(1),
+    MERGE_TABLE_REGIONS_PREPARE(1),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
+     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
      */
-    DISPATCH_MERGING_REGIONS_PRE_OPERATION(2),
+    MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(2),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
      */
-    DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS(3),
+    MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(3),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
+     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
      */
-    DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS(4),
+    MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(4),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
+     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
      */
-    DISPATCH_MERGING_REGIONS_POST_OPERATION(5),
-    ;
-
+    MERGE_TABLE_REGIONS_CLOSE_REGIONS(5),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_PREPARE = 1;</code>
+     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
      */
-    public static final int DISPATCH_MERGING_REGIONS_PREPARE_VALUE = 1;
+    MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(6),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_PRE_OPERATION = 2;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
      */
-    public static final int DISPATCH_MERGING_REGIONS_PRE_OPERATION_VALUE = 2;
+    MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(7),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
      */
-    public static final int DISPATCH_MERGING_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
+    MERGE_TABLE_REGIONS_UPDATE_META(8),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS = 4;</code>
+     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
      */
-    public static final int DISPATCH_MERGING_REGIONS_DO_MERGE_IN_RS_VALUE = 4;
+    MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(9),
     /**
-     * <code>DISPATCH_MERGING_REGIONS_POST_OPERATION = 5;</code>
+     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
      */
-    public static final int DISPATCH_MERGING_REGIONS_POST_OPERATION_VALUE = 5;
-
-
-    public final int getNumber() {
-      return value;
-    }
-
+    MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(10),
     /**
-     * @deprecated Use {@link #forNumber(int)} instead.
+     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
      */
-    @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)
-  }
+    MERGE_TABLE_REGIONS_POST_OPERATION(11),
+    ;
 
-  /**
-   * Protobuf enum {@code hbase.pb.SplitTableRegionState}
-   */
-  public enum SplitTableRegionState
-      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
-     */
-    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>
+     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
      */
-    SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS(8),
+    public static final int MERGE_TABLE_REGIONS_PREPARE_VALUE = 1;
     /**
-     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 9;</code>
+     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 2;</code>
      */
-    SPLIT_TABLE_REGION_POST_OPERATION(9),
-    ;
-
+    public static final int MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 2;
     /**
-     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 3;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PREPARE_VALUE = 1;
+    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION_VALUE = 3;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
+     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 4;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_VALUE = 2;
+    public static final int MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE_VALUE = 4;
     /**
-     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 3;</code>
+     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 5;</code>
      */
-    public static final int SPLIT_TABLE_REGION_CLOSE_PARENT_REGION_VALUE = 3;
+    public static final int MERGE_TABLE_REGIONS_CLOSE_REGIONS_VALUE = 5;
     /**
-     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 4;</code>
+     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 6;</code>
      */
-    public static final int SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS_VALUE = 4;
+    public static final int MERGE_TABLE_REGIONS_CREATE_MERGED_REGION_VALUE = 6;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 5;</code>
+     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 7;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR_VALUE = 5;
+    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION_VALUE = 7;
     /**
-     * <code>SPLIT_TABLE_REGION_UPDATE_META = 6;</code>
+     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 8;</code>
      */
-    public static final int SPLIT_TABLE_REGION_UPDATE_META_VALUE = 6;
+    public static final int MERGE_TABLE_REGIONS_UPDATE_META_VALUE = 8;
     /**
-     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 7;</code>
+     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 9;</code>
      */
-    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR_VALUE = 7;
+    public static final int MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION_VALUE = 9;
     /**
-     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 8;</code>
+     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 10;</code>
      */
-    public static final int SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS_VALUE = 8;
+    public static final int MERGE_TABLE_REGIONS_OPEN_MERGED_REGION_VALUE = 10;
     /**
-     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 9;</code>
+     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 11;</code>
      */
-    public static final int SPLIT_TABLE_REGION_POST_OPERATION_VALUE = 9;
+    public static final int MERGE_TABLE_REGIONS_POST_OPERATION_VALUE = 11;
 
 
     public final int getNumber() {
@@ -1939,34 +1838,36 @@ 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_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;
+        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;
         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);
             }
           };
 
@@ -1980,12 +1881,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(14);
     }
 
-    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(
@@ -1996,116 +1897,100 @@ 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)
   }
 
   /**
-   * Protobuf enum {@code hbase.pb.MergeTableRegionsState}
+   * Protobuf enum {@code hbase.pb.SplitTableRegionState}
    */
-  public enum MergeTableRegionsState
+  public enum SplitTableRegionState
       implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
     /**
-     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
-     */
-    MERGE_TABLE_REGIONS_PREPARE(1),
-    /**
-     * <code>MERGE_TABLE_REGIONS_PRE_OPERATION = 2;</code>
-     */
-    MERGE_TABLE_REGIONS_PRE_OPERATION(2),
-    /**
-     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
      */
-    MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS(3),
+    SPLIT_TABLE_REGION_PREPARE(1),
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 4;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
      */
-    MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION(4),
+    SPLIT_TABLE_REGION_PRE_OPERATION(2),
     /**
-     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 5;</code>
+     * <code>SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;</code>
      */
-    MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE(5),
+    SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE(3),
     /**
-     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 6;</code>
+     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
      */
-    MERGE_TABLE_REGIONS_CLOSE_REGIONS(6),
+    SPLIT_TABLE_REGION_CLOSE_PARENT_REGION(4),
     /**
-     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 7;</code>
+     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
      */
-    MERGE_TABLE_REGIONS_CREATE_MERGED_REGION(7),
+    SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS(5),
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 8;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;</code>
      */
-    MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION(8),
+    SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR(6),
     /**
-     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 9;</code>
+     * <code>SPLIT_TABLE_REGION_UPDATE_META = 7;</code>
      */
-    MERGE_TABLE_REGIONS_UPDATE_META(9),
+    SPLIT_TABLE_REGION_UPDATE_META(7),
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 10;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;</code>
      */
-    MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION(10),
+    SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR(8),
     /**
-     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 11;</code>
+     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;</code>
      */
-    MERGE_TABLE_REGIONS_OPEN_MERGED_REGION(11),
+    SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS(9),
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 12;</code>
+     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 10;</code>
      */
-    MERGE_TABLE_REGIONS_POST_OPERATION(12),
+    SPLIT_TABLE_REGION_POST_OPERATION(10),
     ;
 
     /**
-     * <code>MERGE_TABLE_REGIONS_PREPARE = 1;</code>
-     */
-    public static final int MERGE_TABLE_REGIONS_PREPARE_VALUE = 1;
-    /**
-     * <code>MERGE_TABLE_REGIONS_PRE_OPERATION = 2;</code>
-     */
-    public static final int MERGE_TABLE_REGIONS_PRE_OPERATION_VALUE = 2;
-    /**
-     * <code>MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS = 3;</code>
+     * <code>SPLIT_TABLE_REGION_PREPARE = 1;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_MOVE_REGION_TO_SAME_RS_VALUE = 3;
+    public static final int SPLIT_TABLE_REGION_PREPARE_VALUE = 1;
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION = 4;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION = 2;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_OPERATION_VALUE = 4;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_VALUE = 2;
     /**
-     * <code>MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE = 5;</code>
+     * <code>SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE = 3;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_SET_MERGING_TABLE_STATE_VALUE = 5;
+    public static final int SPLIT_TABLE_REGION_SET_SPLITTING_TABLE_STATE_VALUE = 3;
     /**
-     * <code>MERGE_TABLE_REGIONS_CLOSE_REGIONS = 6;</code>
+     * <code>SPLIT_TABLE_REGION_CLOSE_PARENT_REGION = 4;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_CLOSE_REGIONS_VALUE = 6;
+    public static final int SPLIT_TABLE_REGION_CLOSE_PARENT_REGION_VALUE = 4;
     /**
-     * <code>MERGE_TABLE_REGIONS_CREATE_MERGED_REGION = 7;</code>
+     * <code>SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS = 5;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_CREATE_MERGED_REGION_VALUE = 7;
+    public static final int SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS_VALUE = 5;
     /**
-     * <code>MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION = 8;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR = 6;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_PRE_MERGE_COMMIT_OPERATION_VALUE = 8;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_PONR_VALUE = 6;
     /**
-     * <code>MERGE_TABLE_REGIONS_UPDATE_META = 9;</code>
+     * <code>SPLIT_TABLE_REGION_UPDATE_META = 7;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_UPDATE_META_VALUE = 9;
+    public static final int SPLIT_TABLE_REGION_UPDATE_META_VALUE = 7;
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION = 10;</code>
+     * <code>SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR = 8;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_POST_MERGE_COMMIT_OPERATION_VALUE = 10;
+    public static final int SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_PONR_VALUE = 8;
     /**
-     * <code>MERGE_TABLE_REGIONS_OPEN_MERGED_REGION = 11;</code>
+     * <code>SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS = 9;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_OPEN_MERGED_REGION_VALUE = 11;
+    public static final int SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS_VALUE = 9;
     /**
-     * <code>MERGE_TABLE_REGIONS_POST_OPERATION = 12;</code>
+     * <code>SPLIT_TABLE_REGION_POST_OPERATION = 10;</code>
      */
-    public static final int MERGE_TABLE_REGIONS_POST_OPERATION_VALUE = 12;
+    public static final int SPLIT_TABLE_REGION_POST_OPERATION_VALUE = 10;
 
 
     public final int getNumber() {
@@ -2116,37 +2001,35 @@ 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_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;
+        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;
         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);
             }
           };
 
@@ -2160,12 +2043,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(16);
+      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(
@@ -2176,11 +2059,11 @@ 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)
   }
 
   /**
@@ -2321,7 +2204,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(17);
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(16);
     }
 
     private static final ServerCrashState[] VALUES = values();
@@ -2344,443 +2227,56 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.ServerCrashState)
   }
 
-  /**
-   * Protobuf enum {@code hbase.pb.RegionTransitionState}
-   */
-  public enum RegionTransitionState
-      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+  public interface CreateTableStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableStateData)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
     /**
-     * <code>REGION_TRANSITION_QUEUE = 1;</code>
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
      */
-    REGION_TRANSITION_QUEUE(1),
+    boolean hasUserInfo();
     /**
-     * <code>REGION_TRANSITION_DISPATCH = 2;</code>
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
      */
-    REGION_TRANSITION_DISPATCH(2),
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation getUserInfo();
     /**
-     * <code>REGION_TRANSITION_FINISH = 3;</code>
+     * <code>required .hbase.pb.UserInformation user_info = 1;</code>
      */
-    REGION_TRANSITION_FINISH(3),
-    ;
+    org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
 
     /**
-     * <code>REGION_TRANSITION_QUEUE = 1;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
      */
-    public static final int REGION_TRANSITION_QUEUE_VALUE = 1;
+    boolean hasTableSchema();
     /**
-     * <code>REGION_TRANSITION_DISPATCH = 2;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
      */
-    public static final int REGION_TRANSITION_DISPATCH_VALUE = 2;
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
     /**
-     * <code>REGION_TRANSITION_FINISH = 3;</code>
+     * <code>required .hbase.pb.TableSchema table_schema = 2;</code>
      */
-    public static final int REGION_TRANSITION_FINISH_VALUE = 3;
-
-
-    public final int getNumber() {
-      return value;
-    }
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
 
     /**
-     * @deprecated Use {@link #forNumber(int)} instead.
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
-    @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 {
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
     /**
-     * <code>MOVE_REGION_UNASSIGN = 1;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
-    MOVE_REGION_UNASSIGN(1),
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
     /**
-     * <code>MOVE_REGION_ASSIGN = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
-    MOVE_REGION_ASSIGN(2),
-    ;
-
+    int getRegionInfoCount();
     /**
-     * <code>MOVE_REGION_UNASSIGN = 1;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
-    public static final int MOVE_REGION_UNASSIGN_VALUE = 1;
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
     /**
-     * <code>MOVE_REGION_ASSIGN = 2;</code>
-     */
-    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)
-  }
-
-  /**
-   * Protobuf enum {@code hbase.pb.GCRegionState}
-   */
-  public enum GCRegionState
-      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>GC_REGION_PREPARE = 1;</code>
-     */
-    GC_REGION_PREPARE(1),
-    /**
-     * <code>GC_REGION_ARCHIVE = 2;</code>
-     */
-    GC_REGION_ARCHIVE(2),
-    /**
-     * <code>GC_REGION_PURGE_METADATA = 3;</code>
-     */
-    GC_REGION_PURGE_METADATA(3),
-    ;
-
-    /**
-     * <code>GC_REGION_PREPARE = 1;</code>
-     */
-    public static final int GC_REGION_PREPARE_VALUE = 1;
-    /**
-     * <code>GC_REGION_ARCHIVE = 2;</code>
-     */
-    public static final int GC_REGION_ARCHIVE_VALUE = 2;
-    /**
-     * <code>GC_REGION_PURGE_METADATA = 3;</code>
-     */
-    public static final int GC_REGION_PURGE_METADATA_VALUE = 3;
-
-
-    public final int getNumber() {
-      return value;
-    }
-
-    /**
-     * @deprecated Use {@link #forNumber(int)} instead.
-     */
-    @java.lang.Deprecated
-    public static GCRegionState valueOf(int value) {
-      return forNumber(value);
-    }
-
-    public static GCRegionState forNumber(int value) {
-      switch (value) {
-        case 1: return GC_REGION_PREPARE;
-        case 2: return GC_REGION_ARCHIVE;
-        case 3: return GC_REGION_PURGE_METADATA;
-        default: return null;
-      }
-    }
-
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCRegionState>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
-        GCRegionState> internalValueMap =
-          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCRegionState>() {
-            public GCRegionState findValueByNumber(int number) {
-              return GCRegionState.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(20);
-    }
-
-    private static final GCRegionState[] VALUES = values();
-
-    public static GCRegionState 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 GCRegionState(int value) {
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.GCRegionState)
-  }
-
-  /**
-   * Protobuf enum {@code hbase.pb.GCMergedRegionsState}
-   */
-  public enum GCMergedRegionsState
-      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>GC_MERGED_REGIONS_PREPARE = 1;</code>
-     */
-    GC_MERGED_REGIONS_PREPARE(1),
-    /**
-     * <code>GC_MERGED_REGIONS_PURGE = 2;</code>
-     */
-    GC_MERGED_REGIONS_PURGE(2),
-    /**
-     * <code>GC_REGION_EDIT_METADATA = 3;</code>
-     */
-    GC_REGION_EDIT_METADATA(3),
-    ;
-
-    /**
-     * <code>GC_MERGED_REGIONS_PREPARE = 1;</code>
-     */
-    public static final int GC_MERGED_REGIONS_PREPARE_VALUE = 1;
-    /**
-     * <code>GC_MERGED_REGIONS_PURGE = 2;</code>
-     */
-    public static final int GC_MERGED_REGIONS_PURGE_VALUE = 2;
-    /**
-     * <code>GC_REGION_EDIT_METADATA = 3;</code>
-     */
-    public static final int GC_REGION_EDIT_METADATA_VALUE = 3;
-
-
-    public final int getNumber() {
-      return value;
-    }
-
-    /**
-     * @deprecated Use {@link #forNumber(int)} instead.
-     */
-    @java.lang.Deprecated
-    public static GCMergedRegionsState valueOf(int value) {
-      return forNumber(value);
-    }
-
-    public static GCMergedRegionsState forNumber(int value) {
-      switch (value) {
-        case 1: return GC_MERGED_REGIONS_PREPARE;
-        case 2: return GC_MERGED_REGIONS_PURGE;
-        case 3: return GC_REGION_EDIT_METADATA;
-        default: return null;
-      }
-    }
-
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCMergedRegionsState>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
-        GCMergedRegionsState> internalValueMap =
-          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCMergedRegionsState>() {
-            public GCMergedRegionsState findValueByNumber(int number) {
-              return GCMergedRegionsState.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(21);
-    }
-
-    private static final GCMergedRegionsState[] VALUES = values();
-
-    public static GCMergedRegionsState 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 GCMergedRegionsState(int value) {
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:hbase.pb.GCMergedRegionsState)
-  }
-
-  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>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
      */
     org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
         int index);
@@ -17225,7 +16721,7 @@ public final class MasterProcedureProtos {
        * <code>required .hbase.pb.SnapshotDescription snapshot = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescriptionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescriptionOrBuilder>
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
           snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -19275,7 +18771,7 @@ public final class MasterProcedureProtos {
        * <code>required .hbase.pb.SnapshotDescription snapshot = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescriptionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescriptionOrBuilder>
           getSnapshotFieldBuilder() {
         if (snapshotBuilder_ == null) {
           snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -20414,8 +19910,8 @@ public final class MasterProcedureProtos {
 
   }
 
-  public interface DispatchMergingRegionsStateDataOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsStateData)
+  public interface MergeTableRegionsStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsStateData)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
@@ -20432,63 +19928,63 @@ public final class MasterProcedureProtos {
     org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
 
     /**
-     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
-    boolean hasTableName();
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
     /**
-     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
     /**
-     * <code>required .hbase.pb.TableName table_name = 2;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
-
+    int getRegionInfoCount();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> 
-        getRegionInfoList();
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
      */
-    int getRegionInfoCount();
+    boolean hasMergedRegionInfo();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
-        getRegionInfoOrBuilderList();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedRegionInfo();
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>required .hbase.pb.RegionInfo merged_region_info = 3;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
-        int index);
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedRegionInfoOrBuilder();
 
     /**
-     * <code>optional bool forcible = 4;</code>
+     * <code>optional bool forcible = 4 [default = false];</code>
      */
     boolean hasForcible();
     /**
-     * <code>optional bool forcible = 4;</code>
+     * <code>optional bool forcible = 4 [default = false];</code>
      */
     boolean getForcible();
   }
   /**
-   * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
+   * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
    */
-  public  static final class DispatchMergingRegionsStateData extends
+  public  static final class MergeTableRegionsStateData extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@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) {
+      // @@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) {
       super(builder);
     }
-    private DispatchMergingRegionsStateData() {
+    private MergeTableRegionsStateData() {
       regionInfo_ = java.util.Collections.emptyList();
       forcible_ = false;
     }
@@ -20498,7 +19994,7 @@ public final class MasterProcedureProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private DispatchMergingRegionsStateData(
+    private MergeTableRegionsStateData(
         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 {
@@ -20535,25 +20031,25 @@ public final class MasterProcedureProtos {
               break;
             }
             case 18: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = tableName_.toBuilder();
-              }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(tableName_);
-                tableName_ = subBuilder.buildPartial();
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000002;
               }
-              bitField0_ |= 0x00000002;
+              regionInfo_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
               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;
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = mergedRegionInfo_.toBuilder();
               }
-              regionInfo_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              mergedRegionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(mergedRegionInfo_);
+                mergedRegionInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
               break;
             }
             case 32: {
@@ -20569,7 +20065,7 @@ public final class MasterProcedureProtos {
         throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
             e).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
           regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
         }
         this.unknownFields = unknownFields.build();
@@ -20578,14 +20074,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_DispatchMergingRegionsStateData_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_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_DispatchMergingRegionsStateData_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
     }
 
     private int bitField0_;
@@ -20610,72 +20106,72 @@ public final class MasterProcedureProtos {
       return userInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance() : userInfo_;
     }
 
-    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;
+    public static final int REGION_INFO_FIELD_NUMBER = 2;
     private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
     public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
       return regionInfo_;
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</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 = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</code>
      */
     public int getRegionInfoCount() {
       return regionInfo_.size();
     }
     /**
-     * <code>repeated .hbase.pb.RegionInfo region_info = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</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 = 3;</code>
+     * <code>repeated .hbase.pb.RegionInfo region_info = 2;</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;</code>
+     * <code>optional bool forcible = 4 [default = false];</code>
      */
     public boolean hasForcible() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional bool forcible = 4;</code>
+     * <code>optional bool forcible = 4 [default = false];</code>
      */
     public boolean getForcible() {
       return forcible_;
@@ -20691,7 +20187,7 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasTableName()) {
+      if (!hasMergedRegionInfo()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -20699,16 +20195,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;
     }
@@ -20718,11 +20214,11 @@ public final class MasterProcedureProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeMessage(1, getUserInfo());
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, getTableName());
-      }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        output.writeMessage(3, regionInfo_.get(i));
+        output.writeMessage(2, regionInfo_.get(i));
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(3, getMergedRegionInfo());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeBool(4, forcible_);
@@ -20739,13 +20235,13 @@ public final class MasterProcedureProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeMessageSize(1, getUserInfo());
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+      for (int i = 0; i < regionInfo_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, getTableName());
+          .computeMessageSize(2, regionInfo_.get(i));
       }
-      for (int i = 0; i < regionInfo_.size(); i++) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, regionInfo_.get(i));
+          .computeMessageSize(3, getMergedRegionInfo());
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
@@ -20762,10 +20258,10 @@ public final class MasterProcedureProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) obj;
 
       boolean result = true;
       result = result && (hasUserInfo() == other.hasUserInfo());
@@ -20773,13 +20269,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()
@@ -20800,14 +20296,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(
@@ -20818,58 +20314,58 @@ public final class MasterProcedureProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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 {
@@ -20881,7 +20377,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.DispatchMergingRegionsStateData prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -20896,25 +20392,25 @@ public final class MasterProcedureProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.DispatchMergingRegionsStateData}
+     * Protobuf type {@code hbase.pb.MergeTableRegionsStateData}
      */
     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.DispatchMergingRegionsStateData)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateDataOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsStateData)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateDataOrBuilder {
       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_DispatchMergingRegionsStateData_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_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_DispatchMergingRegionsStateData_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -20928,8 +20424,8 @@ public final class MasterProcedureProtos {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
-          getTableNameFieldBuilder();
           getRegionInfoFieldBuilder();
+          getMergedRegionInfoFieldBuilder();
         }
       }
       public Builder clear() {
@@ -20940,18 +20436,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_ & ~0x00000004);
+          bitField0_ = (bitField0_ & ~0x00000002);
         } else {
           regionInfoBuilder_.clear();
         }
+        if (mergedRegionInfoBuilder_ == null) {
+          mergedRegionInfo_ = null;
+        } else {
+          mergedRegionInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         forcible_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
         return this;
@@ -20959,23 +20455,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_DispatchMergingRegionsStateData_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
       }
 
-      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 getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      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);
+      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);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -20986,23 +20482,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_ & 0x00000004) == 0x00000004)) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
             regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
-            bitField0_ = (bitField0_ & ~0x00000004);
+            bitField0_ = (bitField0_ & ~0x00000002);
           }
           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;
         }
@@ -21039,27 +20535,24 @@ 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.DispatchMergingRegionsStateData) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DispatchMergingRegionsStateData)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);
         } else {
           super.mergeFrom(other);
           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;
+      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;
         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_ & ~0x00000004);
+              bitField0_ = (bitField0_ & ~0x00000002);
             } else {
               ensureRegionInfoIsMutable();
               regionInfo_.addAll(other.regionInfo_);
@@ -21072,7 +20565,7 @@ public final class MasterProcedureProtos {
               regionInfoBuilder_.dispose();
               regionInfoBuilder_ = null;
               regionInfo_ = other.regionInfo_;
-              bitField0_ = (bitField0_ & ~0x00000004);
+              bitField0_ = (bitField0_ & ~0x00000002);
               regionInfoBuilder_ = 
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
@@ -21081,6 +20574,9 @@ public final class MasterProcedureProtos {
             }
           }
         }
+        if (other.hasMergedRegionInfo()) {
+          mergeMergedRegionInfo(other.getMergedRegionInfo());
+        }
         if (other.hasForcible()) {
           setForcible(other.getForcible());
         }
@@ -21093,20 +20589,20 @@ public final class MasterProcedureProtos {
         if (!hasUserInfo()) {
           return false;
         }
-        if (!hasTableName()) {
+        if (!hasMergedRegionInfo()) {
           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;
       }
 
@@ -21114,11 +20610,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.DispatchMergingRegionsStateData parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData 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.DispatchMergingRegionsStateData) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsStateData) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -21247,130 +20743,12 @@ 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(

<TRUNCATED>

[02/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8872c63..c5c6484 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,7 +22,6 @@ 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;
@@ -31,19 +30,18 @@ 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 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 testModifyTable() throws Exception {
@@ -210,7 +208,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = ModifyTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // Validate descriptor
     HTableDescriptor currentHtd = UTIL.getAdmin().getTableDescriptor(tableName);
@@ -247,7 +246,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
       new ModifyTableProcedure(procExec.getEnvironment(), htd));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = ModifyTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // 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 = 0; // failing at pre operation
+    int numberOfSteps = 1; // 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 = 0; // failing at pre operation
+    int numberOfSteps = 1; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // cf2 should not be present

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 47b1248..e6e90ef 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,16 +18,12 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
 import java.util.Random;
+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.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -46,19 +42,17 @@ 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/a3c5a744/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 bed8b4f..9141e0f 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,17 +18,11 @@
 
 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;
@@ -53,7 +47,6 @@ 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;
@@ -63,8 +56,6 @@ 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");
@@ -211,7 +202,8 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
       new RestoreSnapshotProcedure(procExec.getEnvironment(), snapshotHTD, snapshot));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = RestoreSnapshotState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     resetProcExecutorTestingKillFlag();
     validateSnapshotRestore();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8cee4d8..c6968d4 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,45 +19,48 @@
 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.conf.Configuration;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-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.HMaster;
-import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
 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.Threads;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 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;
 
-@Category({MasterTests.class, LargeTests.class})
+/**
+ * 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)
 public class TestServerCrashProcedure {
-  private static final Log LOG = LogFactory.getLog(TestServerCrashProcedure.class);
-
-  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);
+  // 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 final HBaseTestingUtility util = new HBaseTestingUtility();
+
   @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);
@@ -68,27 +71,15 @@ 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();
   }
 
-
-  @Test(timeout=60000)
-  public void testCrashTargetRs() throws Exception {
-  }
-
-  @Test(timeout=60000)
-  @Ignore // Fix for AMv2
-  public void testRecoveryAndDoubleExecutionOnRsWithMeta() throws Exception {
-    testRecoveryAndDoubleExecution(true);
-  }
-
-  @Test(timeout=60000)
-  @Ignore // Fix for AMv2
-  public void testRecoveryAndDoubleExecutionOnRsWithoutMeta() throws Exception {
-    testRecoveryAndDoubleExecution(false);
+  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);
   }
 
   /**
@@ -96,49 +87,43 @@ public class TestServerCrashProcedure {
    * needed state.
    * @throws Exception
    */
-  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 {
+  @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)) {
       // 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]);
-      final int count = util.countRows(t);
-      assertTrue("expected some rows", count > 0);
-      final String checksum = util.checksumRows(t);
+      int count = util.countRows(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.
-      final HMaster master = this.util.getHBaseCluster().getMaster();
+      HMaster master = this.util.getHBaseCluster().getMaster();
       final ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
       master.setServerCrashProcessingEnabled(false);
-      // 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);
+      // 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);
       // 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(rsToKill);
+      master.getServerManager().moveFromOnlineToDeadServers(hrs.getServerName());
       // Enable test flags and then queue the crash procedure.
       ProcedureTestingUtility.waitNoProcedureRunning(procExec);
       ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
-      long procId = procExec.submitProcedure(new ServerCrashProcedure(
-          procExec.getEnvironment(), rsToKill, true, carryingMeta));
+      long procId =
+        procExec.submitProcedure(new ServerCrashProcedure(
+          procExec.getEnvironment(), hrs.getServerName(), 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/a3c5a744/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
new file mode 100644
index 0000000..c3b910e
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSplitTableRegionProcedure.java
@@ -0,0 +1,420 @@
+/**
+ * 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/a3c5a744/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 f7b4100..f453a67 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,7 +23,6 @@ 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;
@@ -76,10 +75,6 @@ public abstract class TestTableDDLProcedureBase {
   }
 
   protected ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
-    return getMaster().getMasterProcedureExecutor();
-  }
-
-  protected HMaster getMaster() {
-    return UTIL.getHBaseCluster().getMaster();
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 22583d3..6d9475f 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,12 +18,8 @@
 
 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;
@@ -38,7 +34,6 @@ 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;
@@ -46,8 +41,6 @@ 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();
@@ -178,7 +171,9 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
       new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    // 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);
 
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
     UTIL.waitUntilAllRegionsAssigned(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 bc7f32a..2fb4741 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
@@ -28,10 +28,8 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -42,6 +40,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -56,9 +55,9 @@ 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.client.Table;
+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.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
@@ -67,10 +66,13 @@ import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 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.TableNamespaceManager;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
@@ -78,7 +80,6 @@ import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 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.hbase.util.Threads;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -88,6 +89,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestRule;
 
+import com.google.common.collect.Sets;
+
 @Category(MediumTests.class)
 public class TestNamespaceAuditor {
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
@@ -311,10 +314,19 @@ public class TestNamespaceAuditor {
       shouldFailMerge = fail;
     }
 
+    private boolean triggered = false;
+
+    public synchronized void waitUtilTriggered() throws InterruptedException {
+      while (!triggered) {
+        wait();
+      }
+    }
+
     @Override
     public synchronized void preMergeRegionsAction(
         final ObserverContext<MasterCoprocessorEnvironment> ctx,
         final HRegionInfo[] regionsToMerge) throws IOException {
+      triggered = true;
       notifyAll();
       if (shouldFailMerge) {
         throw new IOException("fail merge");
@@ -325,16 +337,16 @@ public class TestNamespaceAuditor {
   @Test
   public void testRegionMerge() throws Exception {
     String nsp1 = prefix + "_regiontest";
-    final int initialRegions = 3;
     NamespaceDescriptor nspDesc =
         NamespaceDescriptor.create(nsp1)
-            .addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "" + initialRegions)
+            .addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "3")
             .addConfiguration(TableNamespaceManager.KEY_MAX_TABLES, "2").build();
     ADMIN.createNamespace(nspDesc);
     final TableName tableTwo = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table2");
     byte[] columnFamily = Bytes.toBytes("info");
     HTableDescriptor tableDescOne = new HTableDescriptor(tableTwo);
     tableDescOne.addFamily(new HColumnDescriptor(columnFamily));
+    final int initialRegions = 3;
     ADMIN.createTable(tableDescOne, Bytes.toBytes("1"), Bytes.toBytes("2000"), initialRegions);
     Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
     try (Table table = connection.getTable(tableTwo)) {
@@ -342,41 +354,102 @@ public class TestNamespaceAuditor {
     }
     ADMIN.flush(tableTwo);
     List<HRegionInfo> hris = ADMIN.getTableRegions(tableTwo);
-    assertEquals(initialRegions, hris.size());
     Collections.sort(hris);
-    Future<?> f = ADMIN.mergeRegionsAsync(
+    // merge the two regions
+    final Set<String> encodedRegionNamesToMerge =
+        Sets.newHashSet(hris.get(0).getEncodedName(), hris.get(1).getEncodedName());
+    ADMIN.mergeRegionsAsync(
       hris.get(0).getEncodedNameAsBytes(),
       hris.get(1).getEncodedNameAsBytes(),
       false);
-    f.get(10, TimeUnit.SECONDS);
+    UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        RegionStates regionStates =
+            UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+        for (HRegionInfo hri : ADMIN.getTableRegions(tableTwo)) {
+          if (encodedRegionNamesToMerge.contains(hri.getEncodedName())) {
+            return false;
+          }
+          if (!regionStates.isRegionInState(hri, RegionState.State.OPEN)) {
+            return false;
+          }
+        }
+        return true;
+      }
 
+      @Override
+      public String explainFailure() throws Exception {
+        RegionStates regionStates =
+            UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+        for (HRegionInfo hri : ADMIN.getTableRegions(tableTwo)) {
+          if (encodedRegionNamesToMerge.contains(hri.getEncodedName())) {
+            return hri + " which is expected to be merged is still online";
+          }
+          if (!regionStates.isRegionInState(hri, RegionState.State.OPEN)) {
+            return hri + " is still in not opened";
+          }
+        }
+        return "Unknown";
+      }
+    });
     hris = ADMIN.getTableRegions(tableTwo);
     assertEquals(initialRegions - 1, hris.size());
     Collections.sort(hris);
+
+    final HRegionInfo hriToSplit = hris.get(1);
     ADMIN.split(tableTwo, Bytes.toBytes("500"));
-    // Not much we can do here until we have split return a Future.
-    Threads.sleep(5000);
+
+    UTIL.waitFor(10000, 100, new Waiter.ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        RegionStates regionStates =
+            UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+        for (HRegionInfo hri : ADMIN.getTableRegions(tableTwo)) {
+          if (hri.getEncodedName().equals(hriToSplit.getEncodedName())) {
+            return false;
+          }
+          if (!regionStates.isRegionInState(hri, RegionState.State.OPEN)) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        RegionStates regionStates =
+            UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+        for (HRegionInfo hri : ADMIN.getTableRegions(tableTwo)) {
+          if (hri.getEncodedName().equals(hriToSplit.getEncodedName())) {
+            return hriToSplit + " which is expected to be split is still online";
+          }
+          if (!regionStates.isRegionInState(hri, RegionState.State.OPEN)) {
+            return hri + " is still in not opened";
+          }
+        }
+        return "Unknown";
+      }
+    });
     hris = ADMIN.getTableRegions(tableTwo);
     assertEquals(initialRegions, hris.size());
     Collections.sort(hris);
 
-    // Fail region merge through Coprocessor hook
+    // fail region merge through Coprocessor hook
     MiniHBaseCluster cluster = UTIL.getHBaseCluster();
     MasterCoprocessorHost cpHost = cluster.getMaster().getMasterCoprocessorHost();
     Coprocessor coprocessor = cpHost.findCoprocessor(CPMasterObserver.class.getName());
     CPMasterObserver masterObserver = (CPMasterObserver) coprocessor;
     masterObserver.failMerge(true);
+    masterObserver.triggered = false;
 
-    f = ADMIN.mergeRegionsAsync(
+    ADMIN.mergeRegionsAsync(
       hris.get(1).getEncodedNameAsBytes(),
       hris.get(2).getEncodedNameAsBytes(),
       false);
-    try {
-      f.get(10, TimeUnit.SECONDS);
-      fail("Merge was supposed to fail!");
-    } catch (ExecutionException ee) {
-      // Expected.
-    }
+    masterObserver.waitUtilTriggered();
     hris = ADMIN.getTableRegions(tableTwo);
     assertEquals(initialRegions, hris.size());
     Collections.sort(hris);
@@ -388,6 +461,67 @@ public class TestNamespaceAuditor {
     assertEquals(initialRegions, ADMIN.getTableRegions(tableTwo).size());
   }
 
+  @Test
+  public void testRegionOperations() throws Exception {
+    String nsp1 = prefix + "_regiontest";
+    NamespaceDescriptor nspDesc = NamespaceDescriptor.create(nsp1)
+        .addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "2")
+        .addConfiguration(TableNamespaceManager.KEY_MAX_TABLES, "2").build();
+    ADMIN.createNamespace(nspDesc);
+    boolean constraintViolated = false;
+    final TableName tableOne = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table1");
+    byte[] columnFamily = Bytes.toBytes("info");
+    HTableDescriptor tableDescOne = new HTableDescriptor(tableOne);
+    tableDescOne.addFamily(new HColumnDescriptor(columnFamily));
+    NamespaceTableAndRegionInfo stateInfo;
+    try {
+      ADMIN.createTable(tableDescOne, Bytes.toBytes("1"), Bytes.toBytes("1000"), 7);
+    } catch (Exception exp) {
+      assertTrue(exp instanceof DoNotRetryIOException);
+      LOG.info(exp);
+      constraintViolated = true;
+    } finally {
+      assertTrue(constraintViolated);
+    }
+    assertFalse(ADMIN.tableExists(tableOne));
+    // This call will pass.
+    ADMIN.createTable(tableDescOne);
+    Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
+    Table htable = connection.getTable(tableOne);
+    UTIL.loadNumericRows(htable, Bytes.toBytes("info"), 1, 1000);
+    ADMIN.flush(tableOne);
+    stateInfo = getNamespaceState(nsp1);
+    assertEquals(1, stateInfo.getTables().size());
+    assertEquals(1, stateInfo.getRegionCount());
+    restartMaster();
+
+    HRegion actualRegion = UTIL.getHBaseCluster().getRegions(tableOne).get(0);
+    CustomObserver observer = (CustomObserver) actualRegion.getCoprocessorHost().findCoprocessor(
+        CustomObserver.class.getName());
+    assertNotNull(observer);
+
+    ADMIN.split(tableOne, Bytes.toBytes("500"));
+    observer.postSplit.await();
+    assertEquals(2, ADMIN.getTableRegions(tableOne).size());
+    actualRegion = UTIL.getHBaseCluster().getRegions(tableOne).get(0);
+    observer = (CustomObserver) actualRegion.getCoprocessorHost().findCoprocessor(
+      CustomObserver.class.getName());
+    assertNotNull(observer);
+
+    //Before we go on split, we should remove all reference store files.
+    ADMIN.compact(tableOne);
+    observer.postCompact.await();
+
+    ADMIN.split(tableOne, getSplitKey(actualRegion.getRegionInfo().getStartKey(),
+      actualRegion.getRegionInfo().getEndKey()));
+    observer.postSplit.await();
+    // Make sure no regions have been added.
+    List<HRegionInfo> hris = ADMIN.getTableRegions(tableOne);
+    assertEquals(2, hris.size());
+
+    htable.close();
+  }
+
   /*
    * Create a table and make sure that the table creation fails after adding this table entry into
    * namespace quota cache. Now correct the failure and recreate the table with same name.
@@ -457,9 +591,16 @@ public class TestNamespaceAuditor {
   }
 
   public static class CustomObserver implements RegionObserver {
+    volatile CountDownLatch postSplit;
     volatile CountDownLatch postCompact;
 
     @Override
+    public void postCompleteSplit(ObserverContext<RegionCoprocessorEnvironment> ctx)
+        throws IOException {
+      postSplit.countDown();
+    }
+
+    @Override
     public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
                             Store store, StoreFile resultFile) throws IOException {
       postCompact.countDown();
@@ -467,6 +608,7 @@ public class TestNamespaceAuditor {
 
     @Override
     public void start(CoprocessorEnvironment e) throws IOException {
+      postSplit = new CountDownLatch(1);
       postCompact = new CountDownLatch(1);
     }
   }
@@ -587,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";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8eb2e58..296b38f 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 = "simple_test";
+  public static final String SIMPLE_SIGNATURE = "simle_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/a3c5a744/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 09fb01f..f6dc8c0 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,9 @@ 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(CompactSplit.LARGE_COMPACTION_THREADS, 3);
-    conf.setInt(CompactSplit.SMALL_COMPACTION_THREADS, 4);
-    conf.setInt(CompactSplit.SPLIT_THREADS, 5);
-    conf.setInt(CompactSplit.MERGE_THREADS, 6);
+    conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 3);
+    conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 4);
+    conf.setInt(CompactSplitThread.SPLIT_THREADS, 5);
   }
 
   @After
@@ -115,10 +114,9 @@ public class TestCompactSplitThread {
       assertEquals(5, regionServer.compactSplitThread.getSplitThreadNum());
 
       // change bigger configurations and do online update
-      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);
+      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 4);
+      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 5);
+      conf.setInt(CompactSplitThread.SPLIT_THREADS, 6);
       try {
         regionServer.compactSplitThread.onConfigurationChange(conf);
       } catch (IllegalArgumentException iae) {
@@ -131,10 +129,9 @@ public class TestCompactSplitThread {
       assertEquals(6, regionServer.compactSplitThread.getSplitThreadNum());
 
       // change smaller configurations and do online update
-      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);
+      conf.setInt(CompactSplitThread.LARGE_COMPACTION_THREADS, 2);
+      conf.setInt(CompactSplitThread.SMALL_COMPACTION_THREADS, 3);
+      conf.setInt(CompactSplitThread.SPLIT_THREADS, 4);
       try {
         regionServer.compactSplitThread.onConfigurationChange(conf);
       } catch (IllegalArgumentException iae) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 747fd54..5f4c0aa 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());
-    CompactSplit thread = new CompactSplit(mockServer);
+    CompactSplitThread thread = new CompactSplitThread(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());
-    CompactSplit thread = new CompactSplit(mockServer);
+    CompactSplitThread thread = new CompactSplitThread(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 CompactSplit}
+   * {@link CompactSplitThread}
    * @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());
-    CompactSplit thread = new CompactSplit(mockServer);
+    CompactSplitThread thread = new CompactSplitThread(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"));
-    CompactSplit cst = new CompactSplit(mockServer);
+    CompactSplitThread cst = new CompactSplitThread(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/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
index 430aef5..4264863 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCorruptedRegionStoreFile.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.util.TestTableName;
 
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -55,10 +54,6 @@ import org.junit.experimental.categories.Category;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-/**
- * HBASE-13651 Handle StoreFileScanner FileNotFoundException
- */
-@Ignore
 @Category({MasterTests.class, LargeTests.class})
 public class TestCorruptedRegionStoreFile {
   private static final Log LOG = LogFactory.getLog(TestCorruptedRegionStoreFile.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 0aa39f6..88bbffb 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().hasRegionsInTransition()) {
+      while (TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
+          .isRegionsInTransition()) {
         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()
-          .hasRegionsInTransition()) {
+          .isRegionsInTransition()) {
         Thread.sleep(200);
         LOG.debug("Waiting on table to finish schema altering");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 dcb2c86..c04f2d4 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,20 +56,19 @@ 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.HMaster;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
 import org.apache.hadoop.hbase.master.RegionState;
-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.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+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;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 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.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
@@ -79,7 +78,6 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -87,6 +85,8 @@ import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
 import com.google.common.base.Joiner;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestRegionMergeTransactionOnCluster {
@@ -154,16 +154,24 @@ public class TestRegionMergeTransactionOnCluster {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     AssignmentManager am = cluster.getMaster().getAssignmentManager();
     RegionStates regionStates = am.getRegionStates();
+    long start = EnvironmentEdgeManager.currentTime();
+    while (!regionStates.isRegionInState(hri, State.MERGED)) {
+      assertFalse("Timed out in waiting one merged region to be in state MERGED",
+        EnvironmentEdgeManager.currentTime() - start > 60000);
+      Thread.sleep(500);
+    }
 
     // We should not be able to assign it again
     am.assign(hri, true);
     assertFalse("Merged region can't be assigned",
       regionStates.isRegionInTransition(hri));
+    assertTrue(regionStates.isRegionInState(hri, State.MERGED));
 
     // We should not be able to unassign it either
-    am.unassign(hri);
+    am.unassign(hri, null);
     assertFalse("Merged region can't be unassigned",
       regionStates.isRegionInTransition(hri));
+    assertTrue(regionStates.isRegionInState(hri, State.MERGED));
 
     table.close();
   }
@@ -200,7 +208,8 @@ public class TestRegionMergeTransactionOnCluster {
       // Create table and load data.
       Table table = createTableAndLoadData(MASTER, tableName);
       // Merge 1st and 2nd region
-      mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1, INITIAL_REGION_NUM - 1);
+      mergeRegionsAndVerifyRegionNum(MASTER, tableName, 0, 1,
+          INITIAL_REGION_NUM - 1);
       verifyRowCount(table, ROWSIZE);
       table.close();
 
@@ -278,9 +287,6 @@ public class TestRegionMergeTransactionOnCluster {
         cleaned = ADMIN.runCatalogScan();
         LOG.debug("catalog janitor returned " + cleaned);
         Thread.sleep(50);
-        // Cleanup is async so wait till all procedures are done running.
-        ProcedureTestingUtility.waitNoProcedureRunning(
-            TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
       }
       assertFalse(regionAdir.toString(), fs.exists(regionAdir));
       assertFalse(regionBdir.toString(), fs.exists(regionBdir));
@@ -314,13 +320,12 @@ public class TestRegionMergeTransactionOnCluster {
     try {
       // Create table and load data.
       Table table = createTableAndLoadData(MASTER, tableName);
-      AssignmentManager am = MASTER.getAssignmentManager();
-      List<HRegionInfo> regions = am.getRegionStates().getRegionsOfTable(tableName);
+      RegionStates regionStates = MASTER.getAssignmentManager().getRegionStates();
+      List<HRegionInfo> regions = regionStates.getRegionsOfTable(tableName);
       // Fake offline one region
       HRegionInfo a = regions.get(0);
       HRegionInfo b = regions.get(1);
-      am.unassign(b);
-      am.offlineRegion(b);
+      regionStates.regionOffline(a);
       try {
         // Merge offline region. Region a is offline here
         admin.mergeRegionsAsync(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false)
@@ -357,7 +362,7 @@ public class TestRegionMergeTransactionOnCluster {
     }
   }
 
-  @Ignore @Test // DISABLED FOR NOW. DON'T KNOW HOW IT IS SUPPOSED TO WORK.
+  @Test
   public void testMergeWithReplicas() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     // Create table and load data.
@@ -425,16 +430,12 @@ public class TestRegionMergeTransactionOnCluster {
     List<HRegionInfo> tableRegionsInMaster;
     long timeout = System.currentTimeMillis() + waitTime;
     while (System.currentTimeMillis() < timeout) {
-      tableRegionsInMeta =
-          MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tablename);
-      tableRegionsInMaster =
-          master.getAssignmentManager().getRegionStates().getRegionsOfTable(tablename);
-      LOG.info(tableRegionsInMaster);
-      LOG.info(tableRegionsInMeta);
-      int tableRegionsInMetaSize = tableRegionsInMeta.size();
-      int tableRegionsInMasterSize = tableRegionsInMaster.size();
-      if (tableRegionsInMetaSize == expectedRegionNum
-          && tableRegionsInMasterSize == expectedRegionNum) {
+      tableRegionsInMeta = MetaTableAccessor.getTableRegionsAndLocations(
+        TEST_UTIL.getConnection(), tablename);
+      tableRegionsInMaster = master.getAssignmentManager().getRegionStates()
+          .getRegionsOfTable(tablename);
+      if (tableRegionsInMeta.size() == expectedRegionNum
+          && tableRegionsInMaster.size() == expectedRegionNum) {
         break;
       }
       Thread.sleep(250);
@@ -470,13 +471,12 @@ public class TestRegionMergeTransactionOnCluster {
     verifyRowCount(table, ROWSIZE);
     LOG.info("Verified " + table.getName());
 
-    // Sleep here is an ugly hack to allow region transitions to finish
+    // sleep here is an ugly hack to allow region transitions to finish
     long timeout = System.currentTimeMillis() + waitTime;
     List<Pair<HRegionInfo, ServerName>> tableRegions;
     while (System.currentTimeMillis() < timeout) {
       tableRegions = MetaTableAccessor.getTableRegionsAndLocations(
           TEST_UTIL.getConnection(), tablename);
-      LOG.info("Found " + tableRegions.size() + ", expecting " + numRegions * replication);
       if (tableRegions.size() == numRegions * replication)
         break;
       Thread.sleep(250);
@@ -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.getRegionsStateInTransition()) {
+        for (RegionState regionState: regionStates.getRegionsInTransition()) {
           // 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/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 3c03827..e6b1bc5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -17,44 +17,16 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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.CompatibilityFactory;
-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.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Connection;
-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;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-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.*;
+import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -70,6 +42,12 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 @Category({RegionServerTests.class, LargeTests.class})
 public class TestRegionServerMetrics {
   private static final Log LOG = LogFactory.getLog(TestRegionServerMetrics.class);


[21/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
index 1ccf488..2435564 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/QuotaProtos.java
@@ -2210,7 +2210,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota req_num = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReqNumFieldBuilder() {
         if (reqNumBuilder_ == null) {
           reqNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2328,7 +2328,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota req_size = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReqSizeFieldBuilder() {
         if (reqSizeBuilder_ == null) {
           reqSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2446,7 +2446,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota write_num = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getWriteNumFieldBuilder() {
         if (writeNumBuilder_ == null) {
           writeNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2564,7 +2564,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota write_size = 4;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getWriteSizeFieldBuilder() {
         if (writeSizeBuilder_ == null) {
           writeSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2682,7 +2682,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota read_num = 5;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReadNumFieldBuilder() {
         if (readNumBuilder_ == null) {
           readNumBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2800,7 +2800,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota read_size = 6;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getReadSizeFieldBuilder() {
         if (readSizeBuilder_ == null) {
           readSizeBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -3486,7 +3486,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.TimedQuota timed_quota = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.TimedQuotaOrBuilder>
           getTimedQuotaFieldBuilder() {
         if (timedQuotaBuilder_ == null) {
           timedQuotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -4244,7 +4244,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.Throttle throttle = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Throttle.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.ThrottleOrBuilder>
           getThrottleFieldBuilder() {
         if (throttleBuilder_ == null) {
           throttleBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -4362,7 +4362,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota space = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getSpaceFieldBuilder() {
         if (spaceBuilder_ == null) {
           spaceBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -6077,7 +6077,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuota quota = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaOrBuilder>
           getQuotaFieldBuilder() {
         if (quotaBuilder_ == null) {
           quotaBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -7351,7 +7351,7 @@ public final class QuotaProtos {
        * <code>optional .hbase.pb.SpaceQuotaStatus quota_status = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatus.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaStatusOrBuilder>
           getQuotaStatusFieldBuilder() {
         if (quotaStatusBuilder_ == null) {
           quotaStatusBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -7856,7 +7856,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes>
         getSizesList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
@@ -7869,7 +7869,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
         getSizesOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
@@ -8540,7 +8540,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.TableName table_name = 1;</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> 
+            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<
@@ -8644,7 +8644,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
         getSizesOrBuilderList() {
       return sizes_;
     }
@@ -8951,7 +8951,7 @@ public final class QuotaProtos {
               sizesBuilder_ = null;
               sizes_ = other.sizes_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              sizesBuilder_ = 
+              sizesBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSizesFieldBuilder() : null;
             } else {
@@ -9187,7 +9187,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
            getSizesOrBuilderList() {
         if (sizesBuilder_ != null) {
           return sizesBuilder_.getMessageOrBuilderList();
@@ -9213,12 +9213,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaRegionSizesResponse.RegionSizes sizes = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder>
            getSizesBuilderList() {
         return getSizesFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizes.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesResponse.RegionSizesOrBuilder>
           getSizesFieldBuilder() {
         if (sizesBuilder_ == null) {
           sizesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -9660,7 +9660,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot>
         getSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
@@ -9673,7 +9673,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
         getSnapshotsOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
@@ -10377,7 +10377,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.TableName table_name = 1;</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> 
+            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<
@@ -10495,7 +10495,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
          */
         private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder> 
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder>
             getSnapshotFieldBuilder() {
           if (snapshotBuilder_ == null) {
             snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -10567,7 +10567,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
         getSnapshotsOrBuilderList() {
       return snapshots_;
     }
@@ -10874,7 +10874,7 @@ public final class QuotaProtos {
               snapshotsBuilder_ = null;
               snapshots_ = other.snapshots_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              snapshotsBuilder_ = 
+              snapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getSnapshotsFieldBuilder() : null;
             } else {
@@ -11110,7 +11110,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
            getSnapshotsOrBuilderList() {
         if (snapshotsBuilder_ != null) {
           return snapshotsBuilder_.getMessageOrBuilderList();
@@ -11136,12 +11136,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot snapshots = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder>
            getSnapshotsBuilderList() {
         return getSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshotOrBuilder>
           getSnapshotsFieldBuilder() {
         if (snapshotsBuilder_ == null) {
           snapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -11583,7 +11583,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot>
         getTableSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
@@ -11596,7 +11596,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
         getTableSnapshotsOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
@@ -11607,7 +11607,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot>
         getNsSnapshotsList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
@@ -11620,7 +11620,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
         getNsSnapshotsOrBuilderList();
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
@@ -12329,7 +12329,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.TableName table_name = 1;</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> 
+            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<
@@ -12447,7 +12447,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
          */
         private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder> 
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder>
             getSnapshotFieldBuilder() {
           if (snapshotBuilder_ == null) {
             snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -12642,7 +12642,7 @@ public final class QuotaProtos {
         if (ref instanceof java.lang.String) {
           return (java.lang.String) ref;
         } else {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
               (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
           java.lang.String s = bs.toStringUtf8();
           if (bs.isValidUtf8()) {
@@ -12658,7 +12658,7 @@ public final class QuotaProtos {
           getNamespaceBytes() {
         java.lang.Object ref = namespace_;
         if (ref instanceof java.lang.String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
               org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           namespace_ = b;
@@ -13040,7 +13040,7 @@ public final class QuotaProtos {
             getNamespaceBytes() {
           java.lang.Object ref = namespace_;
           if (ref instanceof String) {
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                     (java.lang.String) ref);
             namespace_ = b;
@@ -13190,7 +13190,7 @@ public final class QuotaProtos {
          * <code>optional .hbase.pb.SpaceQuotaSnapshot snapshot = 2;</code>
          */
         private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder> 
+            org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuotaSnapshotOrBuilder>
             getSnapshotFieldBuilder() {
           if (snapshotBuilder_ == null) {
             snapshotBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -13262,7 +13262,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
         getTableSnapshotsOrBuilderList() {
       return tableSnapshots_;
     }
@@ -13297,7 +13297,7 @@ public final class QuotaProtos {
     /**
      * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
         getNsSnapshotsOrBuilderList() {
       return nsSnapshots_;
     }
@@ -13633,7 +13633,7 @@ public final class QuotaProtos {
               tableSnapshotsBuilder_ = null;
               tableSnapshots_ = other.tableSnapshots_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              tableSnapshotsBuilder_ = 
+              tableSnapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getTableSnapshotsFieldBuilder() : null;
             } else {
@@ -13659,7 +13659,7 @@ public final class QuotaProtos {
               nsSnapshotsBuilder_ = null;
               nsSnapshots_ = other.nsSnapshots_;
               bitField0_ = (bitField0_ & ~0x00000002);
-              nsSnapshotsBuilder_ = 
+              nsSnapshotsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getNsSnapshotsFieldBuilder() : null;
             } else {
@@ -13895,7 +13895,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
            getTableSnapshotsOrBuilderList() {
         if (tableSnapshotsBuilder_ != null) {
           return tableSnapshotsBuilder_.getMessageOrBuilderList();
@@ -13921,12 +13921,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.TableQuotaSnapshot table_snapshots = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder>
            getTableSnapshotsBuilderList() {
         return getTableSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.TableQuotaSnapshotOrBuilder>
           getTableSnapshotsFieldBuilder() {
         if (tableSnapshotsBuilder_ == null) {
           tableSnapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -14135,7 +14135,7 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
            getNsSnapshotsOrBuilderList() {
         if (nsSnapshotsBuilder_ != null) {
           return nsSnapshotsBuilder_.getMessageOrBuilderList();
@@ -14161,12 +14161,12 @@ public final class QuotaProtos {
       /**
        * <code>repeated .hbase.pb.GetQuotaStatesResponse.NamespaceQuotaSnapshot ns_snapshots = 2;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder>
            getNsSnapshotsBuilderList() {
         return getNsSnapshotsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshot.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse.NamespaceQuotaSnapshotOrBuilder>
           getNsSnapshotsFieldBuilder() {
         if (nsSnapshotsBuilder_ == null) {
           nsSnapshotsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -14230,97 +14230,97 @@ public final class QuotaProtos {
 
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TimedQuota_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TimedQuota_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Throttle_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Throttle_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_ThrottleRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_ThrottleRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Quotas_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Quotas_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_QuotaUsage_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_QuotaUsage_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceQuota_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceQuota_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceLimitRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceLimitRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceQuotaStatus_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceQuotaStatus_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SpaceQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SpaceQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaRegionSizesResponse_RegionSizes_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_TableQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetSpaceQuotaSnapshotsResponse_TableQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_TableQuotaSnapshot_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetQuotaStatesResponse_NamespaceQuotaSnapshot_fieldAccessorTable;
 


[13/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 49124ea..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ /dev/null
@@ -1,381 +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.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 detail).
- * <p>Procedures move from the REGION_TRANSITION_QUEUE state when they are
- * first submitted, to the REGION_TRANSITION_DISPATCH state when the request
- * to remote server is sent and the Procedure is suspended waiting on external
- * event to be woken again. Once the external event is triggered, Procedure
- * moves to the REGION_TRANSITION_FINISH state.
- */
-@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 HRegionInfo regionInfo;
-  private volatile boolean lock = false;
-
-  public RegionTransitionProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  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() {
-    HRegionInfo hri = getRegionInfo();
-    return hri != null? hri.getTable(): null;
-  }
-
-  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() == null? null: getRegionInfo().getEncodedName());
-  }
-
-  public RegionStateNode getRegionState(final MasterProcedureEnv env) {
-    return env.getAssignmentManager().getRegionStates().
-        getOrCreateRegionNode(getRegionInfo());
-  }
-
-  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;
-
-  /**
-   * Called when the Procedure is in the REGION_TRANSITION_DISPATCH state.
-   * In here we do the RPC call to OPEN/CLOSE the region. The suspending of
-   * the thread so it sleeps until it gets update that the OPEN/CLOSE has
-   * succeeded is complicated. Read the implementations to learn more.
-   */
-  protected abstract boolean updateTransition(MasterProcedureEnv env, RegionStateNode regionNode)
-    throws IOException, ProcedureSuspendedException;
-
-  protected abstract void finishTransition(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());
-    String msg = exception.getMessage() == null? exception.getClass().getSimpleName():
-      exception.getMessage();
-    LOG.warn("Failed " + this + "; " + regionNode.toShortString() + "; exception=" + msg);
-    remoteCallFailed(env, regionNode, exception);
-    // NOTE: This call to wakeEvent puts this Procedure back on the scheduler.
-    // Thereafter, another Worker can be in here so DO NOT MESS WITH STATE beyond
-    // this method. Just get out of this current processing quickly.
-    env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
-  }
-
-  /**
-   * Be careful! At the end of this method, the procedure has either succeeded
-   * and this procedure has been set into a suspended state OR, we failed and
-   * this procedure has been put back on the scheduler ready for another worker
-   * to pick it up. In both cases, we need to exit the current Worker processing
-   * toute de suite!
-   * @return True if we successfully dispatched the call and false if we failed;
-   * if failed, we need to roll back any setup done for the dispatch.
-   */
-  protected boolean addToRemoteDispatcher(final MasterProcedureEnv env,
-      final ServerName targetServer) {
-    assert targetServer.equals(getRegionState(env).getRegionLocation()) :
-      "targetServer=" + targetServer + " getRegionLocation=" +
-        getRegionState(env).getRegionLocation(); // TODO
-
-    LOG.info("Dispatch " + this + "; " + getRegionState(env).toShortString());
-
-    // Put this procedure into suspended mode to wait on report of state change
-    // from remote regionserver. Means Procedure associated ProcedureEvent is marked not 'ready'.
-    env.getProcedureScheduler().suspendEvent(getRegionState(env).getProcedureEvent());
-
-    // Tricky because this can fail. If it fails need to backtrack on stuff like
-    // the 'suspend' done above -- tricky as the 'wake' requeues us -- and ditto
-    // up in the caller; it needs to undo state changes.
-    if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
-      remoteCallFailed(env, targetServer,
-          new FailedRemoteDispatchException(this + " to " + targetServer));
-      return false;
-    }
-    return true;
-  }
-
-  protected void reportTransition(final MasterProcedureEnv env, final ServerName serverName,
-      final TransitionCode code, final long seqId) throws UnexpectedStateException {
-    final RegionStateNode regionNode = getRegionState(env);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Received report " + code + " seqId=" + seqId + ", " +
-            this + "; " + regionNode.toShortString());
-    }
-    if (!serverName.equals(regionNode.getRegionLocation())) {
-      if (isMeta() && regionNode.getRegionLocation() == null) {
-        regionNode.setRegionLocation(serverName);
-      } else {
-        throw new UnexpectedStateException(String.format(
-          "Unexpected state=%s from server=%s; expected server=%s; %s; %s",
-          code, serverName, regionNode.getRegionLocation(),
-          this, regionNode.toShortString()));
-      }
-    }
-
-    reportTransition(env, regionNode, code, seqId);
-
-    // NOTE: This call adds this procedure back on the scheduler.
-    // This makes it so this procedure can run again. Another worker will take
-    // processing to the next stage. At an extreme, the other worker may run in
-    // parallel so DO  NOT CHANGE any state hereafter! This should be last thing
-    // done in this processing step.
-    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 void toStringState(StringBuilder builder) {
-    super.toStringState(builder);
-    RegionTransitionState ts = this.transitionState;
-    if (!isFinished() && ts != null) {
-      builder.append(":").append(ts);
-    }
-  }
-
-  @Override
-  protected Procedure[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
-    final AssignmentManager am = env.getAssignmentManager();
-    final RegionStateNode regionNode = getRegionState(env);
-    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 + " " + this + "; " + regionNode.toShortString());
-      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 figured it is done or it aborted; check getException()
-              am.removeRegionInTransition(getRegionState(env), this);
-              return null;
-            }
-            transitionState = RegionTransitionState.REGION_TRANSITION_DISPATCH;
-            if (env.getProcedureScheduler().waitEvent(regionNode.getProcedureEvent(), this)) {
-              // Why this suspend? Because we want to ensure Store happens before proceed?
-              throw new ProcedureSuspendedException();
-            }
-            break;
-
-          case REGION_TRANSITION_DISPATCH:
-            // 2. send the request to the target server
-            if (!updateTransition(env, regionNode)) {
-              // The operation figured it is done or it 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
-            finishTransition(env, regionNode);
-            am.removeRegionInTransition(regionNode, this);
-            return null;
-        }
-      } while (retry);
-    } catch (IOException e) {
-      LOG.warn("Retryable error trying to transition: " +
-          this + "; " + regionNode.toShortString(), 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(getRegionState(env), this);
-      return;
-    }
-
-    // There is no rollback for assignment unless we cancel the operation by
-    // dropping/disabling the table.
-    throw new UnsupportedOperationException("Unhandled state " + transitionState +
-        "; there is no rollback for assignment unless we cancel the operation by " +
-        "dropping/disabling the table");
-  }
-
-  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())) {
-      try {
-        LOG.debug(LockState.LOCK_EVENT_WAIT + " pid=" + getProcId() + " " +
-          env.getProcedureScheduler().dumpLocks());
-      } catch (IOException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    this.lock = true;
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegion(this, getRegionInfo());
-    lock = false;
-  }
-
-  @Override
-  protected boolean holdLock(final MasterProcedureEnv env) {
-    return true;
-  }
-
-  @Override
-  protected boolean hasLock(final MasterProcedureEnv env) {
-    return lock;
-  }
-
-  @Override
-  protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
-    // The operation is triggered internally on the server
-    // the client does not know about this procedure.
-    return false;
-  }
-
-  /**
-   * Used by ServerCrashProcedure to see if this Assign/Unassign needs processing.
-   * @return ServerName the Assign or Unassign is going against.
-   */
-  public abstract ServerName getServer(final MasterProcedureEnv env);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index a893783..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ /dev/null
@@ -1,733 +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.assignment;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-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.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.client.TableDescriptor;
-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.AbstractStateMachineRegionProcedure;
-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.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
-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.
- * Takes lock on the parent region.
- * It holds the lock for the life of the procedure.
- */
-@InterfaceAudience.Private
-public class SplitTableRegionProcedure
-    extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
-  private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
-  private Boolean traceEnabled = null;
-  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, regionToSplit);
-
-    checkSplitRow(regionToSplit, splitRow);
-
-    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:
-        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 " + getParentRegion().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(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("pid=" + getProcId() + " failed rollback attempt step " + state +
-          " for splitting the region "
-        + getParentRegion().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.forNumber(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(getRegion()))
-        .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()));
-    setRegion(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(getParentRegion().getShortNameToLog());
-    sb.append(", daughterA=");
-    sb.append(daughter_1_HRI.getShortNameToLog());
-    sb.append(", daughterB=");
-    sb.append(daughter_2_HRI.getShortNameToLog());
-  }
-
-  private HRegionInfo getParentRegion() {
-    return getRegion();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.REGION_SPLIT;
-  }
-
-  private byte[] getSplitRow() {
-    return daughter_2_HRI.getStartKey();
-  }
-
-  private static State [] EXPECTED_SPLIT_STATES = new State [] {State.OPEN, State.CLOSED};
-  /**
-   * 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(getParentRegion());
-    HRegionInfo parentHRI = null;
-    if (node != null) {
-      parentHRI = node.getRegionInfo();
-
-      // Lookup the parent HRI state from the AM, which has the latest updated info.
-      // Protect against the case where concurrent SPLIT requests came in and succeeded
-      // just before us.
-      if (node.isInState(State.SPLIT)) {
-        LOG.info("Split of " + parentHRI + " skipped; state is already SPLIT");
-        return false;
-      }
-      if (parentHRI.isSplit() || parentHRI.isOffline()) {
-        LOG.info("Split of " + parentHRI + " skipped because offline/split.");
-        return false;
-      }
-
-      // expected parent to be online or closed
-      if (!node.isInState(EXPECTED_SPLIT_STATES)) {
-        // We may have SPLIT already?
-        setFailure(new IOException("Split " + parentHRI.getRegionNameAsString() +
-            " FAILED because state=" + node.getState() + "; expected " +
-            Arrays.toString(EXPECTED_SPLIT_STATES)));
-        return false;
-      }
-
-      // Ask the remote regionserver if this region is splittable. If we get an IOE, report it
-      // along w/ the failure so can see why we are not splittable at this time.
-      IOException splittableCheckIOE = null;
-      boolean splittable = false;
-      try {
-        GetRegionInfoResponse response =
-            Util.getRegionInfoResponse(env, node.getRegionLocation(), node.getRegionInfo());
-        splittable = response.hasSplittable() && response.getSplittable();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Splittable=" + splittable + " " + this + " " + node.toShortString());
-        }
-      } catch (IOException e) {
-        splittableCheckIOE = e;
-      }
-      if (!splittable) {
-        IOException e = new IOException(parentHRI.getShortNameToLog() + " NOT splittable");
-        if (splittableCheckIOE != null) e.initCause(splittableCheckIOE);
-        setFailure(e);
-        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("pid=" + getProcId() + " split switch is off! skip split of " + parentHRI);
-      setFailure(new IOException("Split region " +
-          (parentHRI == null? "null": parentHRI.getRegionNameAsString()) +
-          " 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(getParentRegion(), 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(), getTableName());
-    final FileSystem fs = mfs.getFileSystem();
-    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, getParentRegion(), 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("pid=" + getProcId() + " preparing to split " + nbFiles + " storefiles for region " +
-      getParentRegion().getShortNameToLog() + " 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 TableDescriptor 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("pid=" + getProcId() + " split storefiles for region " + getParentRegion().getShortNameToLog() +
-          " 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("pid=" + getProcId() + " splitting started for store file: " +
-          sf.getPath() + " for region: " + getParentRegion());
-    }
-
-    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("pid=" + getProcId() + " splitting complete for store file: " +
-          sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
-    }
-    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 " +
-            getParentRegion().getRegionNameAsString() + " split.");
-      }
-      try {
-        for (Mutation p : metaEntries) {
-          HRegionInfo.parseRegionName(p.getRow());
-        }
-      } catch (IOException e) {
-        LOG.error("pid=" + getProcId() + " row key of mutation from coprocessor 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().markRegionAsSplit(getParentRegion(), 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(getParentRegion());
-  }
-
-  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(getParentRegion(), 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 TableDescriptor 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/a3c5a744/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
deleted file mode 100644
index 126718a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/UnassignProcedure.java
+++ /dev/null
@@ -1,247 +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.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.HConstants;
-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.ServerCrashException;
-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);
-
-  /**
-   * Where to send the unassign RPC.
-   */
-  protected volatile ServerName destinationServer;
-
-  private final AtomicBoolean serverCrashed = new AtomicBoolean(false);
-
-  // TODO: should this be in a reassign procedure?
-  //       ...and keep unassign for 'disable' case?
-  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.REGION_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 {
-    UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
-        .setTransitionState(getTransitionState())
-        .setDestinationServer(ProtobufUtil.toServerName(destinationServer))
-        .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
-    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()) {
-      this.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 " + this + "; " + regionNode.toShortString());
-      return false;
-    }
-
-    // if the server is down, mark the operation as complete
-    if (serverCrashed.get() || !isServerOnline(env, regionNode)) {
-      LOG.info("Server already down: " + this + "; " + regionNode.toShortString());
-      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.
-    if (!addToRemoteDispatcher(env, regionNode.getRegionLocation())) {
-      // If addToRemoteDispatcher fails, it calls #remoteCallFailed which
-      // does all cleanup.
-    }
-
-    // We always return true, even if we fail dispatch because addToRemoteDispatcher
-    // failure processing sets state back to REGION_TRANSITION_QUEUE so we try again;
-    // i.e. return true to keep the Procedure running; it has been reset to startover.
-    return true;
-  }
-
-  @Override
-  protected void finishTransition(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(
-          "Received report 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) {
-    // TODO: Is there on-going rpc to cleanup?
-    if (exception instanceof ServerCrashException) {
-      // This exception comes from ServerCrashProcedure after log splitting.
-      // It is ok to let this procedure go on to complete close now.
-      // This will release lock on this region so the subsequent assign can succeed.
-      try {
-        reportTransition(env, regionNode, TransitionCode.CLOSED, HConstants.NO_SEQNUM);
-      } catch (UnexpectedStateException e) {
-        // Should never happen.
-        throw new RuntimeException(e);
-      }
-    } else 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.
-      LOG.info("Ignoring; waiting on ServerCrashProcedure", exception);
-      // serverCrashed.set(true);
-    } else if (exception instanceof NotServingRegionException) {
-      LOG.info("IS THIS OK? ANY LOGS TO REPLAY; ACTING AS THOUGH ALL GOOD " + regionNode, exception);
-      setTransitionState(RegionTransitionState.REGION_TRANSITION_FINISH);
-    } else {
-      // TODO: kill the server in case we get an exception we are not able to handle
-      LOG.warn("Killing server; unexpected exception; " +
-          this + "; " + regionNode.toShortString() +
-        " exception=" + exception);
-      env.getMasterServices().getServerManager().expireServer(regionNode.getRegionLocation());
-      serverCrashed.set(true);
-    }
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    super.toStringClassDetails(sb);
-    sb.append(", server=").append(this.destinationServer);
-  }
-
-  @Override
-  public ServerName getServer(final MasterProcedureEnv env) {
-    return this.destinationServer;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
deleted file mode 100644
index cb3861a..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
+++ /dev/null
@@ -1,60 +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.assignment;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-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.GetRegionInfoRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
-
-/**
- * Utility for this assignment package only.
- */
-@InterfaceAudience.Private
-class Util {
-  private Util() {}
-
-  /**
-   * Raw call to remote regionserver to get info on a particular region.
-   * @throws IOException Let it out so can report this IOE as reason for failure
-   */
-  static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
-      final ServerName regionLocation, final HRegionInfo hri)
-  throws IOException {
-    // TODO: There is no timeout on this controller. Set one!
-    HBaseRpcController controller = env.getMasterServices().getClusterConnection().
-        getRpcControllerFactory().newController();
-    final AdminService.BlockingInterface admin =
-        env.getMasterServices().getClusterConnection().getAdmin(regionLocation);
-    GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName());
-    try {
-      return admin.getRegionInfo(controller, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 a494ecc..6410375 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
@@ -1,4 +1,4 @@
- /**
+/**
  * 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
@@ -62,11 +62,9 @@ import com.google.common.collect.Lists;
 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.assignment.AssignmentManager} to assign regions
- * in the edge cases. It doesn't provide an implementation of the
- * actual balancing algorithm.
- *
+ * The base class for load balancers. It provides functions used by
+ * {@link org.apache.hadoop.hbase.master.AssignmentManager} to assign regions in the edge cases.
+ * It doesn't provide an implementation of the actual balancing algorithm.
  */
 public abstract class BaseLoadBalancer implements LoadBalancer {
   protected static final int MIN_SERVER_BALANCE = 2;
@@ -204,15 +202,7 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       // Use servername and port as there can be dead servers in this list. We want everything with
       // a matching hostname and port to have the same index.
       for (ServerName sn : clusterState.keySet()) {
-        if (sn == null) {
-          LOG.warn("TODO: Enable TRACE on BaseLoadBalancer. Empty servername); " +
-              "skipping; unassigned regions?");
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("EMPTY SERVERNAME " + clusterState.toString());
-          }
-          continue;
-        }
-        if (serversToIndex.get(sn.getAddress().toString()) == null) {
+        if (serversToIndex.get(sn.getHostAndPort()) == null) {
           serversToIndex.put(sn.getHostAndPort(), numServers++);
         }
         if (!hostsToIndex.containsKey(sn.getHostname())) {
@@ -267,10 +257,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
       int tableIndex = 0, regionIndex = 0, regionPerServerIndex = 0;
 
       for (Entry<ServerName, List<HRegionInfo>> entry : clusterState.entrySet()) {
-        if (entry.getKey() == null) {
-          LOG.warn("SERVERNAME IS NULL, skipping " + entry.getValue());
-          continue;
-        }
         int serverIndex = serversToIndex.get(entry.getKey().getHostAndPort());
 
         // keep the servername if this is the first server name for this hostname
@@ -599,6 +585,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     /**
      * Return true if the placement of region on server would lower the availability
      * of the region in question
+     * @param server
+     * @param region
      * @return true or false
      */
     boolean wouldLowerAvailability(HRegionInfo regionInfo, ServerName serverName) {
@@ -911,11 +899,8 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
           }
         }
         if (leastLoadedServerIndex != -1) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Pick the least loaded server " +
-                servers[leastLoadedServerIndex].getHostname() +
-                " with better locality for region " + regions[region].getShortNameToLog());
-          }
+          LOG.debug("Pick the least loaded server " + servers[leastLoadedServerIndex].getHostname()
+            + " with better locality for region " + regions[region]);
         }
         return leastLoadedServerIndex;
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
index a8e22ce..fd98c9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/FavoredStochasticBalancer.java
@@ -469,10 +469,6 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
     }
   }
 
-  public synchronized List<ServerName> getFavoredNodes(HRegionInfo regionInfo) {
-    return this.fnm.getFavoredNodes(regionInfo);
-  }
-
   /*
    * Generate Favored Nodes for daughters during region split.
    *
@@ -713,12 +709,7 @@ public class FavoredStochasticBalancer extends StochasticLoadBalancer implements
             // No favored nodes, lets unassign.
             LOG.warn("Region not on favored nodes, unassign. Region: " + hri
               + " current: " + current + " favored nodes: " + favoredNodes);
-            try {
-              this.services.getAssignmentManager().unassign(hri);
-            } catch (IOException e) {
-              LOG.warn("Failed unassign", e);
-              continue;
-            }
+            this.services.getAssignmentManager().unassign(hri);
             RegionPlan rp = new RegionPlan(hri, null, null);
             regionPlans.add(rp);
             misplacedRegions++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 907e745..f7e166d 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,6 +23,7 @@ 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;
@@ -38,8 +39,9 @@ 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.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;
 
@@ -147,15 +149,19 @@ 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;
+    }
 
-    // TODO: Should this refresh all the regions or only the ones assigned?
+    Set<HRegionInfo> regions = regionStates.getRegionAssignments().keySet();
     boolean includesUserTables = false;
-    for (final HRegionInfo hri : am.getAssignedRegions()) {
+    for (final HRegionInfo hri : regions) {
       cache.refresh(hri);
       includesUserTables = includesUserTables || !hri.isSystemTable();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 818156d..7e8d696 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,27 +20,28 @@ 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
@@ -53,7 +54,7 @@ import com.google.common.collect.MinMaxPriorityQueue;
  * locations for all Regions in a cluster.
  *
  * <p>This classes produces plans for the
- * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager} to execute.
+ * {@link org.apache.hadoop.hbase.master.AssignmentManager} to execute.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class SimpleLoadBalancer extends BaseLoadBalancer {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 4b96bc6..53db1f2 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
@@ -293,11 +293,9 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
     if (total <= 0 || sumMultiplier <= 0
         || (sumMultiplier > 0 && (total / sumMultiplier) < minCostNeedBalance)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Skipping load balancing because balanced cluster; " + "total cost is " + total
+      LOG.info("Skipping load balancing because balanced cluster; " + "total cost is " + total
           + ", sum multiplier is " + sumMultiplier + " min cost which need balance is "
           + minCostNeedBalance);
-      }
       return false;
     }
     return true;
@@ -1155,11 +1153,11 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
         stats = new double[cluster.numServers];
       }
 
-      for (int i = 0; i < cluster.numServers; i++) {
+      for (int i =0; i < cluster.numServers; i++) {
         stats[i] = 0;
         for (int regionIdx : cluster.regionsPerServer[i]) {
           if (regionIdx == cluster.regionIndexToPrimaryIndex[regionIdx]) {
-            stats[i]++;
+            stats[i] ++;
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 edbba83..512f7e2 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
@@ -232,8 +232,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   }
 
   @Override
-  protected Procedure<MasterProcedureEnv>[] execute(final MasterProcedureEnv env)
-  throws ProcedureSuspendedException {
+  protected Procedure<?>[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
     // Local master locks don't store any state, so on recovery, simply finish this procedure
     // immediately.
     if (recoveredMasterLock) return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index 6ebadb4..03fdaef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -52,8 +52,9 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
   @Override
   public void toStringClassDetails(final StringBuilder sb) {
     sb.append(getClass().getSimpleName());
-    sb.append(", namespace=");
+    sb.append(" (namespace=");
     sb.append(getNamespaceName());
+    sb.append(")");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
deleted file mode 100644
index 41502d4..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
+++ /dev/null
@@ -1,133 +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.OutputStream;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-
-/**
- * Base class for all the Region procedures that want to use a StateMachine.
- * It provides some basic helpers like basic locking, sync latch, and toStringClassDetails().
- * Defaults to holding the lock for the life of the procedure.
- */
-@InterfaceAudience.Private
-public abstract class AbstractStateMachineRegionProcedure<TState>
-    extends AbstractStateMachineTableProcedure<TState> {
-  private HRegionInfo hri;
-  private volatile boolean lock = false;
-
-  public AbstractStateMachineRegionProcedure(final MasterProcedureEnv env,
-      final HRegionInfo hri) {
-    super(env);
-    this.hri = hri;
-  }
-
-  public AbstractStateMachineRegionProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  /**
-   * @return The HRegionInfo of the region we are operating on.
-   */
-  protected HRegionInfo getRegion() {
-    return this.hri;
-  }
-
-  /**
-   * Used when deserializing. Otherwise, DON'T TOUCH IT!
-   */
-  protected void setRegion(final HRegionInfo hri) {
-    this.hri = hri;
-  }
-
-  @Override
-  public TableName getTableName() {
-    return getRegion().getTable();
-  }
-
-  @Override
-  public abstract TableOperationType getTableOperationType();
-
-  @Override
-  public void toStringClassDetails(final StringBuilder sb) {
-    super.toStringClassDetails(sb);
-    sb.append(", region=").append(getRegion().getShortNameToLog());
-  }
-
-  /**
-   * Check whether a table is modifiable - exists and either offline or online with config set
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
-    // Checks whether the table exists
-    if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
-      throw new TableNotFoundException(getTableName());
-    }
-  }
-
-  @Override
-  protected boolean holdLock(MasterProcedureEnv env) {
-    return true;
-  }
-
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
-    if (env.getProcedureScheduler().waitRegions(this, getTableName(), getRegion())) {
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    this.lock = true;
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  protected void releaseLock(final MasterProcedureEnv env) {
-    this.lock = false;
-    env.getProcedureScheduler().wakeRegions(this, getTableName(), getRegion());
-  }
-
-  @Override
-  protected boolean hasLock(final MasterProcedureEnv env) {
-    return this.lock;
-  }
-
-  protected void setFailure(Throwable cause) {
-    super.setFailure(getClass().getSimpleName(), cause);
-  }
-
-  @Override
-  protected void serializeStateData(final OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-    HRegionInfo.convert(getRegion()).writeDelimitedTo(stream);
-  }
-
-  @Override
-  protected void deserializeStateData(final InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-    this.hri = HRegionInfo.convert(HBaseProtos.RegionInfo.parseDelimitedFrom(stream));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 1417159..9f23848 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
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.security.User;
 
 /**
  * Base class for all the Table procedures that want to use a StateMachineProcedure.
- * It provides helpers like basic locking, sync latch, and toStringClassDetails().
+ * It provide some basic helpers like basic locking, sync latch, and basic toStringClassDetails().
  */
 @InterfaceAudience.Private
 public abstract class AbstractStateMachineTableProcedure<TState>
@@ -50,15 +50,11 @@ public abstract class AbstractStateMachineTableProcedure<TState>
     this(env, null);
   }
 
-  /**
-   * @param env Uses this to set Procedure Owner at least.
-   */
   protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env,
       final ProcedurePrepareLatch latch) {
-    if (env != null) {
-      this.user = env.getRequestUser();
-      this.setOwner(user);
-    }
+    this.user = env.getRequestUser();
+    this.setOwner(user);
+
     // used for compatibility with clients without procedures
     // they need a sync TableExistsException, TableNotFoundException, TableNotDisabledException, ...
     this.syncLatch = latch;
@@ -114,4 +110,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/a3c5a744/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 34c1853..7bb2887 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,6 +31,7 @@ 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;
@@ -99,10 +100,7 @@ public class AddColumnFamilyProcedure
         setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
-          addChildProcedure(env.getAssignmentManager()
-            .createReopenProcedures(getRegionInfoList(env)));
-        }
+        reOpenAllRegionsIfTableIsOnline(env);
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -287,8 +285,7 @@ public class AddColumnFamilyProcedure
       env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
       // Make sure regions are opened after table descriptor is updated.
-      //reOpenAllRegionsIfTableIsOnline(env);
-      // TODO: NUKE ROLLBACK!!!!
+      reOpenAllRegionsIfTableIsOnline(env);
     }
   }
 
@@ -305,6 +302,25 @@ 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
@@ -346,8 +362,7 @@ public class AddColumnFamilyProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = env.getAssignmentManager().getRegionStates()
-          .getRegionsOfTable(getTableName());
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 c1d0326..683d840 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
@@ -149,12 +149,10 @@ public class CloneSnapshotProcedure
           setNextState(CloneSnapshotState.CLONE_SNAPSHOT_ASSIGN_REGIONS);
           break;
         case CLONE_SNAPSHOT_ASSIGN_REGIONS:
-          CreateTableProcedure.setEnablingState(env, getTableName());
-          addChildProcedure(env.getAssignmentManager().createAssignProcedures(newRegions));
+          CreateTableProcedure.assignRegions(env, getTableName(), 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_SNAPHOST_RESTORE_ACL);
           break;


[25/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AccessControlProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AccessControlProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AccessControlProtos.java
index e83a7ac..06a4e01 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AccessControlProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AccessControlProtos.java
@@ -1024,7 +1024,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.GlobalPermission global_permission = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GlobalPermissionOrBuilder>
           getGlobalPermissionFieldBuilder() {
         if (globalPermissionBuilder_ == null) {
           globalPermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -1142,7 +1142,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.NamespacePermission namespace_permission = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.NamespacePermissionOrBuilder>
           getNamespacePermissionFieldBuilder() {
         if (namespacePermissionBuilder_ == null) {
           namespacePermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -1260,7 +1260,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TablePermission table_permission = 4;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.TablePermissionOrBuilder>
           getTablePermissionFieldBuilder() {
         if (tablePermissionBuilder_ == null) {
           tablePermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2074,7 +2074,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TableName table_name = 1;</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> 
+          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<
@@ -4130,7 +4130,7 @@ public final class AccessControlProtos {
        * <code>required .hbase.pb.Permission permission = 3;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
           getPermissionFieldBuilder() {
         if (permissionBuilder_ == null) {
           permissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -4198,7 +4198,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions>
         getUserPermissionsList();
     /**
      * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
@@ -4211,7 +4211,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder>
         getUserPermissionsOrBuilderList();
     /**
      * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
@@ -4319,7 +4319,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.Permission permissions = 2;</code>
        */
-      java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission> 
+      java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission>
           getPermissionsList();
       /**
        * <code>repeated .hbase.pb.Permission permissions = 2;</code>
@@ -4332,7 +4332,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.Permission permissions = 2;</code>
        */
-      java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+      java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
           getPermissionsOrBuilderList();
       /**
        * <code>repeated .hbase.pb.Permission permissions = 2;</code>
@@ -4452,7 +4452,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.Permission permissions = 2;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
           getPermissionsOrBuilderList() {
         return permissions_;
       }
@@ -4790,7 +4790,7 @@ public final class AccessControlProtos {
                 permissionsBuilder_ = null;
                 permissions_ = other.permissions_;
                 bitField0_ = (bitField0_ & ~0x00000002);
-                permissionsBuilder_ = 
+                permissionsBuilder_ =
                   org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                      getPermissionsFieldBuilder() : null;
               } else {
@@ -5064,7 +5064,7 @@ public final class AccessControlProtos {
         /**
          * <code>repeated .hbase.pb.Permission permissions = 2;</code>
          */
-        public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+        public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
              getPermissionsOrBuilderList() {
           if (permissionsBuilder_ != null) {
             return permissionsBuilder_.getMessageOrBuilderList();
@@ -5090,12 +5090,12 @@ public final class AccessControlProtos {
         /**
          * <code>repeated .hbase.pb.Permission permissions = 2;</code>
          */
-        public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder> 
+        public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder>
              getPermissionsBuilderList() {
           return getPermissionsFieldBuilder().getBuilderList();
         }
         private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-            org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+            org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
             getPermissionsFieldBuilder() {
           if (permissionsBuilder_ == null) {
             permissionsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -5168,7 +5168,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder>
         getUserPermissionsOrBuilderList() {
       return userPermissions_;
     }
@@ -5480,7 +5480,7 @@ public final class AccessControlProtos {
               userPermissionsBuilder_ = null;
               userPermissions_ = other.userPermissions_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              userPermissionsBuilder_ = 
+              userPermissionsBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getUserPermissionsFieldBuilder() : null;
             } else {
@@ -5716,7 +5716,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder>
            getUserPermissionsOrBuilderList() {
         if (userPermissionsBuilder_ != null) {
           return userPermissionsBuilder_.getMessageOrBuilderList();
@@ -5742,12 +5742,12 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.UsersAndPermissions.UserPermissions user_permissions = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions.Builder>
            getUserPermissionsBuilderList() {
         return getUserPermissionsFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissions.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.UserPermissionsOrBuilder>
           getUserPermissionsFieldBuilder() {
         if (userPermissionsBuilder_ == null) {
           userPermissionsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -6396,7 +6396,7 @@ public final class AccessControlProtos {
        * <code>required .hbase.pb.UserPermission user_permission = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
           getUserPermissionFieldBuilder() {
         if (userPermissionBuilder_ == null) {
           userPermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -7393,7 +7393,7 @@ public final class AccessControlProtos {
        * <code>required .hbase.pb.UserPermission user_permission = 1;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
           getUserPermissionFieldBuilder() {
         if (userPermissionBuilder_ == null) {
           userPermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -8507,7 +8507,7 @@ public final class AccessControlProtos {
        * <code>optional .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> 
+          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<
@@ -8610,7 +8610,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission>
         getUserPermissionList();
     /**
      * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
@@ -8623,7 +8623,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
         getUserPermissionOrBuilderList();
     /**
      * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
@@ -8721,7 +8721,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
         getUserPermissionOrBuilderList() {
       return userPermission_;
     }
@@ -9028,7 +9028,7 @@ public final class AccessControlProtos {
               userPermissionBuilder_ = null;
               userPermission_ = other.userPermission_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              userPermissionBuilder_ = 
+              userPermissionBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getUserPermissionFieldBuilder() : null;
             } else {
@@ -9264,7 +9264,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
            getUserPermissionOrBuilderList() {
         if (userPermissionBuilder_ != null) {
           return userPermissionBuilder_.getMessageOrBuilderList();
@@ -9290,12 +9290,12 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.UserPermission user_permission = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder>
            getUserPermissionBuilderList() {
         return getUserPermissionFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>
           getUserPermissionFieldBuilder() {
         if (userPermissionBuilder_ == null) {
           userPermissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -9364,7 +9364,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.Permission permission = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission> 
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission>
         getPermissionList();
     /**
      * <code>repeated .hbase.pb.Permission permission = 1;</code>
@@ -9377,7 +9377,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.Permission permission = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
         getPermissionOrBuilderList();
     /**
      * <code>repeated .hbase.pb.Permission permission = 1;</code>
@@ -9475,7 +9475,7 @@ public final class AccessControlProtos {
     /**
      * <code>repeated .hbase.pb.Permission permission = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
         getPermissionOrBuilderList() {
       return permission_;
     }
@@ -9782,7 +9782,7 @@ public final class AccessControlProtos {
               permissionBuilder_ = null;
               permission_ = other.permission_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              permissionBuilder_ = 
+              permissionBuilder_ =
                 org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
                    getPermissionFieldBuilder() : null;
             } else {
@@ -10018,7 +10018,7 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.Permission permission = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
            getPermissionOrBuilderList() {
         if (permissionBuilder_ != null) {
           return permissionBuilder_.getMessageOrBuilderList();
@@ -10044,12 +10044,12 @@ public final class AccessControlProtos {
       /**
        * <code>repeated .hbase.pb.Permission permission = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder> 
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder>
            getPermissionBuilderList() {
         return getPermissionFieldBuilder().getBuilderList();
       }
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.PermissionOrBuilder>
           getPermissionFieldBuilder() {
         if (permissionBuilder_ == null) {
           permissionBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
@@ -10931,77 +10931,77 @@ public final class AccessControlProtos {
 
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_Permission_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_Permission_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_TablePermission_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_TablePermission_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_NamespacePermission_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_NamespacePermission_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GlobalPermission_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GlobalPermission_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UserPermission_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UserPermission_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UsersAndPermissions_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UsersAndPermissions_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_UsersAndPermissions_UserPermissions_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_UsersAndPermissions_UserPermissions_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GrantRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GrantRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GrantResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GrantResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RevokeRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RevokeRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_RevokeResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_RevokeResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetUserPermissionsRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetUserPermissionsRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_GetUserPermissionsResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_GetUserPermissionsResponse_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CheckPermissionsRequest_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CheckPermissionsRequest_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_CheckPermissionsResponse_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_CheckPermissionsResponse_fieldAccessorTable;
 


[08/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 ed19dc9..b3ca94d 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
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -25,17 +27,8 @@ import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -79,7 +72,6 @@ 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;
@@ -127,12 +119,12 @@ 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.ClearCompactionQueuesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
+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;
@@ -145,8 +137,6 @@ 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;
@@ -204,6 +194,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MapReduceProtos.ScanMet
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsResponse.TableQuotaSnapshot;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -221,8 +212,6 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Implements the regionserver RPC services.
  */
@@ -1476,6 +1465,36 @@ 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.
    *
@@ -1626,8 +1645,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       if (request.hasCompactionState() && request.getCompactionState()) {
         builder.setCompactionState(region.getCompactionState());
       }
-      builder.setSplittable(region.isSplittable());
-      builder.setMergeable(region.isMergeable());
       builder.setIsRecovering(region.isRecovering());
       return builder.build();
     } catch (IOException ie) {
@@ -1838,11 +1855,8 @@ 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";
-          LOG.warn(error);
-          //regionServer.abort(error);
-          //throw new IOException(error);
-          builder.addOpeningState(RegionOpeningState.OPENED);
-          continue;
+          regionServer.abort(error);
+          throw new IOException(error);
         }
         LOG.info("Open " + region.getRegionNameAsString());
 
@@ -3382,62 +3396,4 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new ServiceException(e);
     }
   }
-
-  @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)
-  // UNUSED AS OF AMv2 PURGE!
-  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);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
index 6c4eca9..295b825 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java
@@ -96,14 +96,6 @@ public interface Region extends ConfigurationObserver {
   /** @return True if region is read only */
   boolean isReadOnly();
 
-  /** @return true if region is splittable */
-  boolean isSplittable();
-
-  /**
-   * @return true if region is mergeable
-   */
-  boolean isMergeable();
-
   /**
    * Return the list of Stores managed by this region
    * <p>Use with caution.  Exposed for use of fixup utilities.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index e95932b..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionMergeRequest.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * 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.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 com.google.common.base.Preconditions;
-
-/**
- * Handles processing region merges. Put in a queue, owned by HRegionServer.
- */
-// UNUSED: REMOVE!!!
-@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/a3c5a744/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 5afa652..54aeaa6 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
@@ -183,6 +183,16 @@ 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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
index 8eb78a2..b347b4b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionUnassigner.java
@@ -48,7 +48,8 @@ class RegionUnassigner {
       return;
     }
     unassigning = true;
-    new Thread("RegionUnassigner." + regionInfo.getEncodedName()) {
+    new Thread("Unassign-" + regionInfo) {
+
       @Override
       public void run() {
         LOG.info("Unassign " + regionInfo.getRegionNameAsString());
@@ -64,4 +65,4 @@ class RegionUnassigner {
       }
     }.start();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 bd59c53..eb9811d 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
@@ -18,16 +18,16 @@
  */
 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.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;
 
 import com.google.common.base.Preconditions;
 
@@ -37,14 +37,14 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 class SplitRequest implements Runnable {
   private static final Log LOG = LogFactory.getLog(SplitRequest.class);
-  private final HRegionInfo parent;
+  private final HRegion 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 = region.getRegionInfo();
+    this.parent = (HRegion)region;
     this.midKey = midKey;
     this.server = hrs;
     this.user = user;
@@ -56,30 +56,67 @@ class SplitRequest implements Runnable {
   }
 
   private void doSplitting() {
+    boolean success = false;
     server.metricsRegionServer.incrSplitRequest();
-    if (user != null && user.getUGI() != null) {
-      user.getUGI().doAs (new PrivilegedAction<Void>() {
-        @Override
-        public Void run() {
-          requestRegionSplit();
-          return null;
+    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);
         }
-      });
-    } else {
-      requestRegionSplit();
-    }
-  }
+      } 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();
+      }
 
-  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 objects
-    // 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());
+      if (success) {
+        server.metricsRegionServer.incrSplitSuccess();
+      }
     }
   }
 
@@ -93,4 +130,4 @@ class SplitRequest implements Runnable {
 
     doSplitting();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
index 3ecc750..5ff7a1e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/CloseRegionHandler.java
@@ -125,4 +125,4 @@ public class CloseRegionHandler extends EventHandler {
         remove(this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 f1e42a6..dca02e4 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());
 
-    List<RegionState> rits = status.getRegionsInTransition();
+    Set<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/a3c5a744/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 8ea7012..d7749c2 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.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", 16));
+        "hbase.hregion.open.and.init.threads.max", 10));
     ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
     .getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
         new ThreadFactory() {
@@ -236,4 +236,24 @@ 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;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 517a0cd..e8069ec 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
@@ -289,8 +289,8 @@ public class WALSplitter {
     this.fileBeingSplit = logfile;
     try {
       long logLength = logfile.getLen();
-      LOG.info("Splitting WAL=" + logPath + ", length=" + logLength +
-          ", distributedLogReplay=" + this.distributedLogReplay);
+      LOG.info("Splitting wal: " + logPath + ", length=" + logLength);
+      LOG.info("DistributedLogReplay = " + this.distributedLogReplay);
       status.setStatus("Opening log file");
       if (reporter != null && !reporter.progress()) {
         progress_failed = true;
@@ -298,7 +298,7 @@ public class WALSplitter {
       }
       in = getReader(logfile, skipErrors, reporter);
       if (in == null) {
-        LOG.warn("Nothing to split in WAL=" + logPath);
+        LOG.warn("Nothing to split in log file " + logPath);
         return true;
       }
       int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
@@ -377,7 +377,7 @@ public class WALSplitter {
       iie.initCause(ie);
       throw iie;
     } catch (CorruptedLogFileException e) {
-      LOG.warn("Could not parse, corrupted WAL=" + logPath, e);
+      LOG.warn("Could not parse, corrupted log file " + logPath, e);
       if (this.csm != null) {
         // Some tests pass in a csm of null.
         this.csm.getSplitLogWorkerCoordination().markCorrupted(rootDir,
@@ -397,7 +397,7 @@ public class WALSplitter {
           in.close();
         }
       } catch (IOException exception) {
-        LOG.warn("Could not close WAL reader: " + exception.getMessage());
+        LOG.warn("Could not close wal reader: " + exception.getMessage());
         LOG.debug("exception details", exception);
       }
       try {
@@ -1595,10 +1595,8 @@ public class WALSplitter {
           if (wap == null) {
             wap = getWriterAndPath(logEntry);
             if (wap == null) {
-              if (LOG.isTraceEnabled()) {
-                // This log spews the full edit. Can be massive in the log. Enable only debugging
-                // WAL lost edit issues.
-                LOG.trace("getWriterAndPath decided we don't need to write edits for " + logEntry);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("getWriterAndPath decided we don't need to write edits for " + logEntry);
               }
               return;
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
index a6a5c17..69cd233 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
@@ -88,8 +88,8 @@ public class RegionServerTracker extends ZooKeeperListener {
               int magicLen = ProtobufUtil.lengthOfPBMagic();
               ProtobufUtil.mergeFrom(rsInfoBuilder, data, magicLen, data.length - magicLen);
             }
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Added tracking of RS " + nodePath);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Added tracking of RS " + nodePath);
             }
           } catch (KeeperException e) {
             LOG.warn("Get Rs info port from ephemeral node", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 5c8b29b..afc070d 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
@@ -86,10 +86,10 @@ 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.HMaster;
+import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -3323,14 +3323,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public void moveRegionAndWait(HRegionInfo destRegion, ServerName destServer)
       throws InterruptedException, IOException {
     HMaster master = getMiniHBaseCluster().getMaster();
-    // TODO: Here we start the move. The move can take a while.
-    getAdmin().move(destRegion.getEncodedNameAsBytes(),
+    getHBaseAdmin().move(destRegion.getEncodedNameAsBytes(),
         Bytes.toBytes(destServer.getServerName()));
     while (true) {
       ServerName serverName = master.getAssignmentManager().getRegionStates()
           .getRegionServerOfRegion(destRegion);
       if (serverName != null && serverName.equals(destServer)) {
-        assertRegionOnServer(destRegion, serverName, 2000);
+        assertRegionOnServer(destRegion, serverName, 200);
         break;
       }
       Thread.sleep(10);
@@ -3995,7 +3994,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
         if (master == null) return false;
         AssignmentManager am = master.getAssignmentManager();
         if (am == null) return false;
-        return !am.hasRegionsInTransition();
+        final RegionStates regionStates = am.getRegionStates();
+        return !regionStates.isRegionsInTransition();
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 a99345b..eefde94 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
@@ -302,6 +302,16 @@ 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/a3c5a744/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 cff1a8d..283d79d 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,18 +21,13 @@ 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;
@@ -42,18 +37,21 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 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;
 
+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)
  */
-@Ignore // This is broken since new RegionServers does proper average of regions
-// and because Master is treated as a regionserver though it hosts two regions only.
 @Category({FlakeyTests.class, LargeTests.class})
 @RunWith(value = Parameterized.class)
 public class TestRegionRebalancing {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 f84d9c2..7b69db4 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,18 +42,23 @@ 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;
@@ -98,7 +103,7 @@ public class TestAdmin1 {
 
   @Before
   public void setUp() throws Exception {
-    this.admin = TEST_UTIL.getAdmin();
+    this.admin = TEST_UTIL.getHBaseAdmin();
   }
 
   @After
@@ -746,7 +751,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
+    admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
     try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
@@ -801,7 +806,7 @@ public class TestAdmin1 {
 
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin = TEST_UTIL.getAdmin();
+    admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
 
@@ -987,7 +992,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)) {
 
@@ -1052,7 +1057,8 @@ 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/a3c5a744/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 607fc61..0014401 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -528,6 +528,8 @@ 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/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index 074a421..04bd224 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
@@ -35,14 +34,14 @@ 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.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 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.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -53,7 +52,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -323,10 +321,17 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc).get();
 
+      // add region to meta.
+      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 region.
       HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
       AssignmentManager am = master.getAssignmentManager();
-      HRegionInfo hri = am.getRegionStates().getRegionsOfTable(tableName).get(0);
+      admin.assign(hri.getRegionName()).get();
+      am.waitForAssignment(hri);
 
       // assert region on server
       RegionStates regionStates = am.getRegionStates();
@@ -335,32 +340,22 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
       assertTrue(regionStates.getRegionState(hri).isOpened());
 
       // Region is assigned now. Let's assign it again.
-      // Master should not abort, and region should stay assigned.
+      // Master should not abort, and region should be assigned.
       admin.assign(hri.getRegionName()).get();
-      try {
-        am.waitForAssignment(hri);
-        fail("Expected NoSuchProcedureException");
-      } catch (NoSuchProcedureException e) {
-        // Expected
-      }
+      am.waitForAssignment(hri);
       assertTrue(regionStates.getRegionState(hri).isOpened());
 
       // unassign region
       admin.unassign(hri.getRegionName(), true).get();
-      try {
-        am.waitForAssignment(hri);
-        fail("Expected NoSuchProcedureException");
-      } catch (NoSuchProcedureException e) {
-        // Expected
-      }
-      assertTrue(regionStates.getRegionState(hri).isClosed());
+      am.waitForAssignment(hri);
+      assertTrue(regionStates.getRegionState(hri).isOpened());
     } finally {
       TEST_UTIL.deleteTable(tableName);
     }
   }
 
   HRegionInfo createTableAndGetOneRegion(final TableName tableName)
-      throws IOException, InterruptedException, ExecutionException {
+      throws IOException, InterruptedException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(FAMILY));
     admin.createTable(desc, Bytes.toBytes("A"), Bytes.toBytes("Z"), 5);
@@ -382,12 +377,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
     }
   }
 
-  @Ignore @Test
-  // Turning off this tests in AMv2. Doesn't make sense.Offlining means something
-  // different now.
-  // You can't 'offline' a region unless you know what you are doing
-  // Will cause the Master to tell the regionserver to shut itself down because
-  // regionserver is reporting the state as OPEN.
+  @Test
   public void testOfflineRegion() throws Exception {
     final TableName tableName = TableName.valueOf("testOfflineRegion");
     try {
@@ -395,6 +385,8 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
 
       RegionStates regionStates =
           TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+      ServerName serverName = regionStates.getRegionServerOfRegion(hri);
+      TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
       admin.offline(hri.getRegionName()).get();
 
       long timeoutTime = System.currentTimeMillis() + 3000;
@@ -450,7 +442,7 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
         if (now > timeoutTime) {
           fail("Failed to move the region in time: " + regionStates.getRegionState(hri));
         }
-        regionStates.wait(50);
+        regionStates.waitForUpdate(50);
       }
     } finally {
       TEST_UTIL.deleteTable(tableName);
@@ -474,9 +466,6 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
             regionServerCount.incrementAndGet();
           });
       Assert.assertEquals(regionServerCount.get(), 2);
-    } catch (Exception e) {
-      LOG.info("Exception", e);
-      throw e;
     } finally {
       TEST_UTIL.deleteTable(tableName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
index 2abc54d..913c2e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
@@ -47,14 +47,12 @@ 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.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
  * Will split the table, and move region randomly when testing.
  */
-@Ignore // Can't move hbase:meta off master server in AMv2. TODO.
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableGetMultiThreaded {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java
index b9f11d5..8743266 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithBasicCompaction.java
@@ -17,14 +17,13 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
-@Ignore // Can't move hbase:meta off master server in AMv2. TODO.
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableGetMultiThreadedWithBasicCompaction extends
     TestAsyncTableGetMultiThreaded {
@@ -33,4 +32,5 @@ public class TestAsyncTableGetMultiThreadedWithBasicCompaction extends
   public static void setUp() throws Exception {
     setUp(MemoryCompactionPolicy.BASIC);
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java
index dd94398..ef75373 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreadedWithEagerCompaction.java
@@ -17,14 +17,13 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
-@Ignore // Can't move hbase:meta off master server in AMv2. TODO.
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableGetMultiThreadedWithEagerCompaction extends
     TestAsyncTableGetMultiThreaded {
@@ -33,4 +32,5 @@ public class TestAsyncTableGetMultiThreadedWithEagerCompaction extends
   public static void setUp() throws Exception {
     setUp(MemoryCompactionPolicy.EAGER);
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index 6e50312..023095f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -593,14 +593,12 @@ public class TestBlockEvictionFromClient {
       put.addColumn(FAMILY, QUALIFIER2, data2);
       table.put(put);
       region.flush(true);
-      LOG.info("About to SPLIT on " + Bytes.toString(ROW1));
       TEST_UTIL.getAdmin().split(tableName, ROW1);
       List<HRegionInfo> tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
       // Wait for splits
       while (tableRegions.size() != 2) {
         tableRegions = TEST_UTIL.getAdmin().getTableRegions(tableName);
         Thread.sleep(100);
-        LOG.info("Waiting on SPLIT to complete...");
       }
       region.compact(true);
       Iterator<CachedBlock> iterator = cache.iterator();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 e1277fa..80b7208 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,15 +18,14 @@
  */
 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;
@@ -37,11 +36,16 @@ 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;
@@ -53,9 +57,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 @Category({ MasterTests.class, MediumTests.class })
 public class TestEnableTable {
@@ -101,17 +105,15 @@ public class TestEnableTable {
     rs.getRegionServer().stop("stop");
     cluster.waitForRegionServerToStop(rs.getRegionServer().getServerName(), 10000);
 
-    // We used to enable the table here but AMv2 would hang waiting on a RS to check-in.
-    // Revisit.
+    LOG.debug("Now enabling table " + tableName);
+
+    admin.enableTable(tableName);
+    assertTrue(admin.isTableEnabled(tableName));
 
     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/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 9eaa716..7f44a2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -677,8 +677,6 @@ public class TestFromClientSide3 {
             });
           fail("This cp should fail because the target lock is blocked by previous put");
         } catch (Throwable ex) {
-          // TODO!!!! Is this right? It catches everything including the above fail
-          // if it happens (which it seems too....)
         }
       });
       cpService.shutdown();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 e99ee07..bfe10b5 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
@@ -18,12 +18,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-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 com.google.common.collect.Lists;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -65,6 +60,7 @@ 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.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -86,7 +82,12 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
-import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * This class is for testing HBaseConnectionManager features
@@ -230,6 +231,8 @@ 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);
   }
 
@@ -967,7 +970,7 @@ public class TestHCM {
    * that we really delete it.
    * @throws Exception
    */
-  @Ignore @Test
+  @Test
   public void testRegionCaching() throws Exception{
     TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAM_NAM).close();
     Configuration conf =  new Configuration(TEST_UTIL.getConfiguration());
@@ -1033,7 +1036,7 @@ public class TestHCM {
     Assert.assertNotNull(curServer.getOnlineRegion(regionName));
     Assert.assertNull(destServer.getOnlineRegion(regionName));
     Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
-        getAssignmentManager().hasRegionsInTransition());
+        getAssignmentManager().getRegionStates().isRegionsInTransition());
 
     // 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.
@@ -1046,7 +1049,7 @@ public class TestHCM {
     while (destServer.getOnlineRegion(regionName) == null ||
         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-        master.getAssignmentManager().hasRegionsInTransition()) {
+        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
       // wait for the move to be finished
       Thread.sleep(1);
     }
@@ -1105,7 +1108,7 @@ public class TestHCM {
     while (curServer.getOnlineRegion(regionName) == null ||
         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-        master.getAssignmentManager().hasRegionsInTransition()) {
+        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
       // wait for the move to be finished
       Thread.sleep(1);
     }
@@ -1290,7 +1293,7 @@ public class TestHCM {
     return prevNumRetriesVal;
   }
 
-  @Ignore @Test
+  @Test
   public void testMulti() throws Exception {
     Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
     try {
@@ -1325,8 +1328,6 @@ public class TestHCM {
       HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(destServerId);
 
       ServerName destServerName = destServer.getServerName();
-      ServerName metaServerName = TEST_UTIL.getHBaseCluster().getServerHoldingMeta();
-      assertTrue(!destServerName.equals(metaServerName));
 
        //find another row in the cur server that is less than ROW_X
       List<Region> regions = curServer.getOnlineRegions(TABLE_NAME3);
@@ -1352,11 +1353,11 @@ public class TestHCM {
       Assert.assertNotNull(curServer.getOnlineRegion(regionName));
       Assert.assertNull(destServer.getOnlineRegion(regionName));
       Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
-          getAssignmentManager().hasRegionsInTransition());
+          getAssignmentManager().getRegionStates().isRegionsInTransition());
 
        // Moving. It's possible that we don't have all the regions online at this point, so
-      //  the test depends only on the region we're looking at.
-      LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString());
+      //  the test must depends only on the region we're looking at.
+      LOG.info("Move starting region="+toMove.getRegionInfo().getRegionNameAsString());
       TEST_UTIL.getAdmin().move(
           toMove.getRegionInfo().getEncodedNameAsBytes(),
           destServerName.getServerName().getBytes()
@@ -1365,7 +1366,7 @@ public class TestHCM {
       while (destServer.getOnlineRegion(regionName) == null ||
           destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
           curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
-          master.getAssignmentManager().hasRegionsInTransition()) {
+          master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
         // wait for the move to be finished
         Thread.sleep(1);
       }
@@ -1477,4 +1478,107 @@ public class TestHCM {
     table.close();
     connection.close();
   }
-}
\ No newline at end of file
+
+  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/a3c5a744/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 3847e6e..a700ebe 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
@@ -29,6 +29,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -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;
@@ -59,15 +60,12 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 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 edu.umd.cs.findbugs.annotations.Nullable;
-
 /**
  * Tests the scenarios where replicas are enabled for the meta table
  */
@@ -107,11 +105,7 @@ public class TestMetaWithReplicas {
     for (int replicaId = 1; replicaId < 3; replicaId ++) {
       HRegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
         replicaId);
-      try {
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().waitForAssignment(h);
-      } catch (NoSuchProcedureException e) {
-        LOG.info("Presume the procedure has been cleaned up so just proceed: " + e.toString());
-      }
+      TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().waitForAssignment(h);
     }
     LOG.debug("All meta replicas assigned");
   }
@@ -262,7 +256,7 @@ public class TestMetaWithReplicas {
     }
   }
 
-  @Ignore @Test // Uses FSCK. Needs fixing after HBASE-14614.
+  @Test
   public void testChangingReplicaCount() throws Exception {
     // tests changing the replica count across master restarts
     // reduce the replica count from 3 to 2
@@ -281,9 +275,6 @@ public class TestMetaWithReplicas {
     assert(metaZnodes.size() == originalReplicaCount); //we should have what was configured before
     TEST_UTIL.getHBaseClusterInterface().getConf().setInt(HConstants.META_REPLICAS_NUM,
         newReplicaCount);
-    if (TEST_UTIL.getHBaseCluster().countServedRegions() < newReplicaCount) {
-      TEST_UTIL.getHBaseCluster().startRegionServer();
-    }
     TEST_UTIL.getHBaseClusterInterface().startMaster(sn.getHostname(), 0);
     TEST_UTIL.getHBaseClusterInterface().waitForActiveAndReadyMaster();
     TEST_UTIL.waitFor(10000, predicateMetaHasReplicas(newReplicaCount));
@@ -340,7 +331,7 @@ public class TestMetaWithReplicas {
     HbckTestingUtil.assertNoErrors(hbck);
   }
 
-  @Ignore @Test // Disabled. Relies on FSCK which needs work for AMv2.
+  @Test
   public void testHBaseFsckWithFewerMetaReplicas() throws Exception {
     ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
         TEST_UTIL.getConfiguration());
@@ -358,7 +349,7 @@ public class TestMetaWithReplicas {
     assertErrors(hbck, new ERROR_CODE[]{});
   }
 
-  @Ignore @Test // The close silently doesn't work any more since HBASE-14614. Fix.
+  @Test
   public void testHBaseFsckWithFewerMetaReplicaZnodes() throws Exception {
     ClusterConnection c = (ClusterConnection)ConnectionFactory.createConnection(
         TEST_UTIL.getConfiguration());
@@ -392,7 +383,7 @@ public class TestMetaWithReplicas {
     fail("Expected TableNotFoundException");
   }
 
-  @Ignore @Test // Disabled. Currently can't move hbase:meta in AMv2.
+  @Test
   public void testMetaAddressChange() throws Exception {
     // checks that even when the meta's location changes, the various
     // caches update themselves. Uses the master operations to test
@@ -420,16 +411,13 @@ public class TestMetaWithReplicas {
     TEST_UTIL.getAdmin().move(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
         Bytes.toBytes(moveToServer.getServerName()));
     int i = 0;
-    assert !moveToServer.equals(currentServer);
-    LOG.info("CurrentServer=" + currentServer + ", moveToServer=" + moveToServer);
-    final int max = 10000;
     do {
       Thread.sleep(10);
       data = ZKUtil.getData(zkw, primaryMetaZnode);
       currentServer = ProtobufUtil.toServerName(data);
       i++;
-    } while (!moveToServer.equals(currentServer) && i < max); //wait for 10 seconds overall
-    assert(i != max);
+    } while (!moveToServer.equals(currentServer) && i < 1000); //wait for 10 seconds overall
+    assert(i != 1000);
     TEST_UTIL.getAdmin().disableTable(tableName);
     assertTrue(TEST_UTIL.getAdmin().isTableDisabled(tableName));
   }
@@ -448,7 +436,7 @@ public class TestMetaWithReplicas {
       int i = 0;
       do {
         LOG.debug("Waiting for the replica " + hrl.getRegionInfo() + " to come up");
-        Thread.sleep(10000); //wait for the detection/recovery
+        Thread.sleep(30000); //wait for the detection/recovery
         rl = conn.locateRegion(TableName.META_TABLE_NAME, Bytes.toBytes(""), false, true);
         hrl = rl.getRegionLocation(1);
         i++;
@@ -457,11 +445,14 @@ public class TestMetaWithReplicas {
     }
   }
 
-  @Ignore @Test // Disabled because fsck and this needs work for AMv2
+  @Test
   public void testHBaseFsckWithExcessMetaReplicas() throws Exception {
     // 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/a3c5a744/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 ef00b24..1b18ee2 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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -44,7 +43,11 @@ 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;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -616,33 +619,34 @@ public class TestScannersFromClientSide {
     byte[] regionName = hri.getRegionName();
     int i = cluster.getServerWith(regionName);
     HRegionServer rs = cluster.getRegionServer(i);
-    LOG.info("Unassigning " + hri);
-    TEST_UTIL.getAdmin().unassign(hri.getRegionName(), true);
+    ProtobufUtil.closeRegion(null,
+      rs.getRSRpcServices(), rs.getServerName(), regionName);
     long startTime = EnvironmentEdgeManager.currentTime();
-    long timeOut = 10000;
-    boolean offline = false;
+    long timeOut = 300000;
     while (true) {
       if (rs.getOnlineRegion(regionName) == null) {
-        offline = true;
         break;
       }
       assertTrue("Timed out in closing the testing region",
         EnvironmentEdgeManager.currentTime() < startTime + timeOut);
+      Thread.sleep(500);
     }
-    assertTrue(offline);
-    LOG.info("Assigning " + hri);
-    TEST_UTIL.getAdmin().assign(hri.getRegionName());
+
+    // 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);
     startTime = EnvironmentEdgeManager.currentTime();
     while (true) {
-      rs = cluster.getRegionServer(cluster.getServerWith(regionName));
-      if (rs != null && rs.getOnlineRegion(regionName) != null) {
-        offline = false;
+      if (rs.getOnlineRegion(regionName) != null) {
         break;
       }
       assertTrue("Timed out in open the testing region",
         EnvironmentEdgeManager.currentTime() < startTime + timeOut);
+      Thread.sleep(500);
     }
-    assertFalse(offline);
 
     // c0:0, c1:1
     kvListExp = new ArrayList<>();


[03/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 6824597..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestSplitTableRegionProcedure.java
+++ /dev/null
@@ -1,428 +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.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.assignment.SplitTableRegionProcedure;
-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
-  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
-  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
-  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
-  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
-  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
-  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
-  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();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index ac199cb..f806472 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -97,7 +96,7 @@ public class TestFavoredStochasticBalancerPickers extends BalancerTestBase {
   }
 
 
-  @Ignore @Test
+  @Test
   public void testPickers() throws Exception {
 
     TableName tableName = TableName.valueOf("testPickers");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 83c9355..ca4337e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -45,8 +45,7 @@ import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -265,7 +263,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
   }
 
-  @Ignore @Test
+  @Test
   public void testMisplacedRegions() throws Exception {
 
     TableName tableName = TableName.valueOf("testMisplacedRegions");
@@ -295,7 +293,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     regionFNMap.put(misplacedRegion, newFavoredNodes);
     fnm.updateFavoredNodes(regionFNMap);
 
-    final RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
     final ServerName current = regionStates.getRegionServerOfRegion(misplacedRegion);
     assertNull("Misplaced region is still hosted on favored node, not expected.",
         FavoredNodesPlan.getFavoredServerPosition(fnm.getFavoredNodes(misplacedRegion), current));
@@ -333,7 +331,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     stopServersAndWaitUntilProcessed(serversToStop);
 
     TEST_UTIL.waitUntilNoRegionsInTransition();
-    final RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
     TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -349,7 +347,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
   }
 
-  @Ignore @Test
+  @Test
   public void testAllFavoredNodesDead() throws Exception {
 
     TableName tableName = TableName.valueOf("testAllFavoredNodesDead");
@@ -367,7 +365,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     // Lets kill all the RS that are favored nodes for this region.
     stopServersAndWaitUntilProcessed(currentFN);
 
-    final RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+    RegionStates regionStates = master.getAssignmentManager().getRegionStates();
     TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -387,8 +385,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     FavoredNodeAssignmentHelper helper = new FavoredNodeAssignmentHelper(serversForNewFN, conf);
     helper.initialize();
 
-    for (RegionStateNode regionState: regionStates.getRegionsInTransition()) {
-      HRegionInfo regionInfo = regionState.getRegionInfo();
+    for (RegionState regionState : regionStates.getRegionsInTransition()) {
+      HRegionInfo regionInfo = regionState.getRegion();
       List<ServerName> newFavoredNodes = helper.generateFavoredNodes(regionInfo);
       assertNotNull(newFavoredNodes);
       assertEquals(FavoredNodeAssignmentHelper.FAVORED_NODES_NUM, newFavoredNodes.size());
@@ -410,7 +408,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     checkFavoredNodeAssignments(tableName, fnm, regionStates);
   }
 
-  @Ignore @Test
+  @Test
   public void testAllFavoredNodesDeadMasterRestarted() throws Exception {
 
     TableName tableName = TableName.valueOf("testAllFavoredNodesDeadMasterRestarted");
@@ -428,8 +426,7 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
     // Lets kill all the RS that are favored nodes for this region.
     stopServersAndWaitUntilProcessed(currentFN);
 
-    final RegionStates regionStatesBeforeMaster =
-        master.getAssignmentManager().getRegionStates();
+    RegionStates regionStatesBeforeMaster = master.getAssignmentManager().getRegionStates();
     TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
@@ -441,8 +438,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
         regionStatesBeforeMaster.getRegionState(region).isFailedOpen());
 
     List<HRegionInfo> rit = Lists.newArrayList();
-    for (RegionStateNode regionState: regionStatesBeforeMaster.getRegionsInTransition()) {
-      HRegionInfo regionInfo = regionState.getRegionInfo();
+    for (RegionState regionState : regionStatesBeforeMaster.getRegionsInTransition()) {
+      HRegionInfo regionInfo = regionState.getRegion();
       LOG.debug("Region in transition after stopping FN's: " + regionInfo);
       rit.add(regionInfo);
       assertTrue("Region: " + regionInfo + " should be RIT",

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index b80bcc8..586f93e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.test.LoadTestKVGenerator;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -179,7 +178,7 @@ public class TestSimpleRegionNormalizerOnCluster {
     admin.deleteTable(TABLENAME);
   }
 
-  @Ignore @Test(timeout = 60000) // TODO: FIX!
+  @Test(timeout = 60000)
   @SuppressWarnings("deprecation")
   public void testRegionNormalizationMergeOnCluster() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
index 822e57c..2b28c9f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureSchedulerPerformanceEvaluation.java
@@ -80,7 +80,7 @@ public class MasterProcedureSchedulerPerformanceEvaluation extends AbstractHBase
 
   private class RegionProcedure extends TestMasterProcedureScheduler.TestRegionProcedure {
     RegionProcedure(long procId, HRegionInfo hri) {
-      super(procId, hri.getTable(), TableOperationType.REGION_UNASSIGN, hri);
+      super(procId, hri.getTable(), TableOperationType.UNASSIGN, hri);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 99dcf1d..ce8b0c6 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,7 +25,6 @@ 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;
@@ -49,11 +48,7 @@ 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;
@@ -64,45 +59,7 @@ import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 public class MasterProcedureTestingUtility {
   private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
 
-  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;
-        }
-      });
+  private MasterProcedureTestingUtility() {
   }
 
   // ==========================================================================
@@ -338,9 +295,6 @@ public class MasterProcedureTestingUtility {
     return put;
   }
 
-  // ==========================================================================
-  //  Procedure Helpers
-  // ==========================================================================
   public static long generateNonceGroup(final HMaster master) {
     return master.getClusterConnection().getNonceGenerator().getNonceGroup();
   }
@@ -364,6 +318,13 @@ 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 {
@@ -375,9 +336,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);
-      restartMasterProcedureExecutor(procExec);
+      ProcedureTestingUtility.restart(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
 
@@ -405,8 +366,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));
-      restartMasterProcedureExecutor(procExec);
+      LOG.info("Restart " + i + " exec state: " + procExec.getProcedure(procId));
+      ProcedureTestingUtility.restart(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
     assertEquals(true, procExec.isRunning());
@@ -438,7 +399,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);
-        restartMasterProcedureExecutor(procExec);
+        ProcedureTestingUtility.restart(procExec);
         ProcedureTestingUtility.waitProcedure(procExec, procId);
       }
     } finally {
@@ -483,7 +444,7 @@ public class MasterProcedureTestingUtility {
     try {
       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
       LOG.info("Restart and rollback procId=" + procId);
-      restartMasterProcedureExecutor(procExec);
+      ProcedureTestingUtility.restart(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     } finally {
       assertTrue(procExec.unregisterListener(abortListener));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 506e537..31eedfc 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,28 +22,26 @@ 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 {
@@ -63,7 +61,8 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf1);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf1);
 
     // Test 2: Add a column family offline
     UTIL.getAdmin().disableTable(tableName);
@@ -72,7 +71,8 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId2);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
   }
 
   @Test(timeout=60000)
@@ -91,7 +91,8 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
 
     // add the column family that exists
     long procId2 = procExec.submitProcedure(
@@ -139,9 +140,11 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = AddColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf4);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf4);
   }
 
   @Test(timeout = 60000)
@@ -161,9 +164,11 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = AddColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf5);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf5);
   }
 
   @Test(timeout = 60000)
@@ -182,9 +187,10 @@ public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
-    int numberOfSteps = 0; // failing at "pre operations"
+    int numberOfSteps = 1; // failing at "pre operations"
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf6);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf6);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 b70aff3..54efe76 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,8 +18,6 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.junit.Assert.assertTrue;
-
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -30,7 +28,6 @@ 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;
@@ -46,6 +43,8 @@ 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);
@@ -148,7 +147,8 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
       new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = CloneSnapshotState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     MasterProcedureTestingUtility.validateTableIsEnabled(
       UTIL.getHBaseCluster().getMaster(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 147ee27..995d98d 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,6 +35,7 @@ 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;
@@ -190,7 +191,8 @@ public class TestCreateNamespaceProcedure {
       new CreateNamespaceProcedure(procExec.getEnvironment(), nsd));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = CreateNamespaceState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // Validate the creation of namespace
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 6bd88c7..c09016c 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,8 +18,6 @@
 
 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;
@@ -42,12 +40,8 @@ import static org.junit.Assert.assertTrue;
 
 @Category({MasterTests.class, MediumTests.class})
 public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
-  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();
+  @Rule
+  public TestName name = new TestName();
 
   @Test(timeout=60000)
   public void testSimpleCreate() throws Exception {
@@ -67,8 +61,9 @@ 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(getMaster(), tableName, regions, F1, F2);
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
   }
 
   @Test(timeout=60000)
@@ -131,21 +126,25 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
       new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
-    MasterProcedureTestingUtility.validateTableCreation(getMaster(), tableName, regions, F1, F2);
+    // 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");
   }
 
   @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);
   }
 
@@ -167,24 +166,11 @@ public class TestCreateTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     TableName tableName = htd.getTableName();
-    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().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/a3c5a744/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 6096755..c4bdc18 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,27 +22,26 @@ 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 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 testDeleteColumnFamily() throws Exception {
@@ -60,7 +59,8 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf1);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf1);
 
     // Test 2: delete the column family that exists offline
     UTIL.getAdmin().disableTable(tableName);
@@ -88,7 +88,8 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf2);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
 
     // delete the column family that does not exist
     long procId2 = procExec.submitProcedure(
@@ -158,9 +159,11 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf4.getBytes()));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = DeleteColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf4);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf4);
   }
 
   @Test(timeout = 60000)
@@ -180,9 +183,11 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = DeleteColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
-    MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf5);
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf5);
   }
 
   @Test(timeout = 60000)
@@ -202,10 +207,10 @@ public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
 
-    int numberOfSteps = 0; // failing at pre operation
+    int numberOfSteps = 1; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
 
     MasterProcedureTestingUtility.validateTableCreation(
-      getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 5ecacb6..acedf1d 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,6 +36,7 @@ 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;
@@ -174,7 +175,8 @@ public class TestDeleteNamespaceProcedure {
       new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = DeleteNamespaceState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     // Validate the deletion of namespace
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 2a11544..431e3e4 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,7 +20,6 @@ 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;
@@ -35,16 +34,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;
 
 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 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, expected=TableNotFoundException.class)
   public void testDeleteNotExistentTable() throws Exception {
@@ -92,7 +90,8 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
 
     // First delete should succeed
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName);
 
     // Second delete should fail with TableNotFound
     ProcedureInfo result = procExec.getResult(procId2);
@@ -127,7 +126,8 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
     long procId = ProcedureTestingUtility.submitAndWait(procExec,
       new DeleteTableProcedure(procExec.getEnvironment(), tableName));
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableDeletion(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName);
   }
 
   @Test(timeout=60000)
@@ -149,8 +149,11 @@ public class TestDeleteTableProcedure extends TestTableDDLProcedureBase {
       new DeleteTableProcedure(procExec.getEnvironment(), tableName));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    // 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.validateTableDeletion(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 11c4e2a..d3fccbe 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,15 +36,13 @@ 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 {
@@ -59,7 +57,8 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 
   @Test(timeout = 60000)
@@ -75,7 +74,8 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId1);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
-    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
 
     // Disable the table again - expect failure
     long procId2 = procExec.submitProcedure(new DisableTableProcedure(
@@ -107,7 +107,8 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId4);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId4);
-    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 
   @Test(timeout=60000)
@@ -127,8 +128,9 @@ public class TestDisableTableProcedure extends TestTableDDLProcedureBase {
       new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
-
-    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
+    int numberOfSteps = DisableTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 01f9ed8..81f2576 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,14 +36,13 @@ 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 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 testEnableTable() throws Exception {
@@ -59,7 +58,8 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
     // Wait the completion
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 
   @Test(timeout=60000, expected=TableNotDisabledException.class)
@@ -113,9 +113,10 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
-
-    MasterProcedureTestingUtility.validateTableIsEnabled(getMaster(), tableName);
+    int numberOfSteps = EnableTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
+    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 
   @Test(timeout = 60000)
@@ -135,8 +136,9 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
 
-    int numberOfSteps = 0; // failing at pre operation
+    int numberOfSteps = 1; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
-    MasterProcedureTestingUtility.validateTableIsDisabled(getMaster(), tableName);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 d2df2bf..af48302 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,6 +19,8 @@
 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;
@@ -29,7 +31,6 @@ 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;
@@ -54,7 +55,6 @@ 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);
+    testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
 
     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);
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
 
     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);
+    testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
 
     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);
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
 
     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);
+    testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
 
     MasterProcedureTestingUtility.validateTableIsEnabled(
         UTIL.getHBaseCluster().getMaster(), tableName);
@@ -297,17 +297,16 @@ public class TestMasterFailoverWithProcedures {
   // ==========================================================================
   //  Test Helpers
   // ==========================================================================
-  public static void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
-      final long procId, final int lastStepBeforeFailover) throws Exception {
+  public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
+      final long procId, final int lastStepBeforeFailover, TState[] states) 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: " + proc);
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
-      MasterProcedureTestingUtility.restartMasterProcedureExecutor(procExec);
+      ProcedureTestingUtility.restart(procExec);
       ProcedureTestingUtility.waitProcedure(procExec, procId);
     }
     ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 b7bc28f..6e85a9f 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().hasRegionsInTransition()) {
+        master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
       Thread.sleep(25);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 5f20c7f..e23c90a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -343,11 +343,11 @@ public class TestMasterProcedureScheduler {
     final HRegionInfo regionA = new HRegionInfo(tableName, Bytes.toBytes("a"), Bytes.toBytes("b"));
 
     queue.addBack(new TestRegionProcedure(1, tableName,
-        TableProcedureInterface.TableOperationType.REGION_ASSIGN, regionA));
+        TableProcedureInterface.TableOperationType.ASSIGN, regionA));
     queue.addBack(new TestTableProcedure(2, tableName,
           TableProcedureInterface.TableOperationType.EDIT));
     queue.addBack(new TestRegionProcedure(3, tableName,
-        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, regionA));
+        TableProcedureInterface.TableOperationType.UNASSIGN, regionA));
 
     // Fetch the 1st item and take the shared lock
     Procedure proc = queue.poll();
@@ -391,13 +391,13 @@ public class TestMasterProcedureScheduler {
     queue.addBack(new TestTableProcedure(1, tableName,
           TableProcedureInterface.TableOperationType.EDIT));
     queue.addBack(new TestRegionProcedure(2, tableName,
-        TableProcedureInterface.TableOperationType.REGION_MERGE, regionA, regionB));
+        TableProcedureInterface.TableOperationType.MERGE, regionA, regionB));
     queue.addBack(new TestRegionProcedure(3, tableName,
-        TableProcedureInterface.TableOperationType.REGION_SPLIT, regionA));
+        TableProcedureInterface.TableOperationType.SPLIT, regionA));
     queue.addBack(new TestRegionProcedure(4, tableName,
-        TableProcedureInterface.TableOperationType.REGION_SPLIT, regionB));
+        TableProcedureInterface.TableOperationType.SPLIT, regionB));
     queue.addBack(new TestRegionProcedure(5, tableName,
-        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, regionC));
+        TableProcedureInterface.TableOperationType.UNASSIGN, regionC));
 
     // Fetch the 1st item and take the write lock
     Procedure proc = queue.poll();
@@ -520,9 +520,9 @@ public class TestMasterProcedureScheduler {
     final HRegionInfo region = new HRegionInfo(tableName, Bytes.toBytes("a"), Bytes.toBytes("b"));
 
     queue.addBack(new TestRegionProcedure(1, tableName,
-        TableProcedureInterface.TableOperationType.REGION_SPLIT, region));
+        TableProcedureInterface.TableOperationType.SPLIT, region));
     queue.addBack(new TestRegionProcedure(1, 2, tableName,
-        TableProcedureInterface.TableOperationType.REGION_UNASSIGN, region));
+        TableProcedureInterface.TableOperationType.UNASSIGN, region));
     queue.addBack(new TestRegionProcedure(3, tableName,
         TableProcedureInterface.TableOperationType.REGION_EDIT, region));
 
@@ -600,7 +600,7 @@ public class TestMasterProcedureScheduler {
     final TestRegionProcedure[] childProcs = new TestRegionProcedure[regions.length];
     for (int i = 0; i < regions.length; ++i) {
       childProcs[i] = new TestRegionProcedure(1, 2 + i, tableName,
-          TableProcedureInterface.TableOperationType.REGION_ASSIGN, regions[i]);
+          TableProcedureInterface.TableOperationType.ASSIGN, regions[i]);
     }
     testInheritedXLockAndChildrenSharedLock(tableName,
       new TestTableProcedure(1, tableName, TableProcedureInterface.TableOperationType.CREATE),
@@ -615,7 +615,7 @@ public class TestMasterProcedureScheduler {
     final TestRegionProcedure[] childProcs = new TestRegionProcedure[regions.length];
     for (int i = 0; i < regions.length; ++i) {
       childProcs[i] = new TestRegionProcedure(1, 2, 3 + i, tableName,
-          TableProcedureInterface.TableOperationType.REGION_ASSIGN, regions[i]);
+          TableProcedureInterface.TableOperationType.ASSIGN, regions[i]);
     }
     testInheritedXLockAndChildrenSharedLock(tableName,
       new TestTableProcedure(1, tableName, TableProcedureInterface.TableOperationType.CREATE),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 f1667ed..184150b 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,6 +28,7 @@ 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;
@@ -124,7 +125,8 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = ModifyColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
         .getMaster(), tableName, cf3, columnDescriptor);
@@ -151,7 +153,8 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = ModifyColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
         .getMaster(), tableName, cf4, columnDescriptor);
@@ -177,7 +180,7 @@ public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
     long procId = procExec.submitProcedure(
       new ModifyColumnFamilyProcedure(procExec.getEnvironment(), tableName, columnDescriptor));
 
-    int numberOfSteps = 0; // failing at pre operation
+    int numberOfSteps = 1; // failing at pre operation
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 5cb117b..1b53d23 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,6 +32,7 @@ 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;
@@ -211,7 +212,8 @@ public class TestModifyNamespaceProcedure {
       new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd));
 
     // Restart the executor and execute the step twice
-    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    int numberOfSteps = ModifyNamespaceState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps);
 
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
     // Validate


[16/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..3a2a6d7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStateStore.java
@@ -0,0 +1,268 @@
+/**
+ * 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/a3c5a744/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
new file mode 100644
index 0000000..dcbf5a4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionStates.java
@@ -0,0 +1,1170 @@
+/**
+ * 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() : 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/a3c5a744/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 6ae9f0f..db0a0e5 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,10 +57,12 @@ 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;
@@ -74,6 +76,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.UpdateFavor
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.RetryCounter;
@@ -311,8 +314,7 @@ public class ServerManager {
     }
   }
 
-  @VisibleForTesting
-  public void regionServerReport(ServerName sn,
+  void regionServerReport(ServerName sn,
       ServerLoad sl) throws YouAreDeadException {
     checkIsDead(sn, "REPORT");
     if (null == this.onlineServers.replace(sn, sl)) {
@@ -612,7 +614,12 @@ public class ServerManager {
       return;
     }
 
-    master.getAssignmentManager().submitServerCrash(serverName, true);
+    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);
 
     // Tell our listeners that a server was removed
     if (!this.listeners.isEmpty()) {
@@ -622,37 +629,6 @@ 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) {
@@ -684,7 +660,9 @@ public class ServerManager {
     }
 
     this.deadservers.add(serverName);
-    master.getAssignmentManager().submitServerCrash(serverName, shouldSplitWal);
+    ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
+    procExec.submitProcedure(new ServerCrashProcedure(
+      procExec.getEnvironment(), serverName, shouldSplitWal, false));
   }
 
   /**
@@ -770,8 +748,9 @@ public class ServerManager {
       throw new IOException("Attempting to send OPEN RPC to server " + server.toString() +
         " failed because no RPC connection found to this server");
     }
-    OpenRegionRequest request =
-        RequestConverter.buildOpenRegionRequest(server, region, favoredNodes, false);
+    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
+      region, favoredNodes,
+      (RecoveryMode.LOG_REPLAY == this.master.getMasterWalManager().getLogRecoveryMode()));
     try {
       OpenRegionResponse response = admin.openRegion(null, request);
       return ResponseConverter.getRegionOpeningState(response);
@@ -853,8 +832,8 @@ public class ServerManager {
         " failed because no RPC connection found to this server");
     }
 
-    OpenRegionRequest request =
-        RequestConverter.buildOpenRegionRequest(server, regionOpenInfos, false);
+    OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server, regionOpenInfos,
+      (RecoveryMode.LOG_REPLAY == this.master.getMasterWalManager().getLogRecoveryMode()));
     try {
       OpenRegionResponse response = admin.openRegion(null, request);
       return ResponseConverter.getRegionOpeningStateList(response);
@@ -898,6 +877,30 @@ 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
@@ -987,7 +990,7 @@ public class ServerManager {
     * @throws IOException
     * @throws RetriesExhaustedException wrapping a ConnectException if failed
     */
-  public AdminService.BlockingInterface getRsAdmin(final ServerName sn)
+  private 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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 2fc2bbb..7017d29 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -710,7 +710,7 @@ public class SplitLogManager {
         long now = EnvironmentEdgeManager.currentTime();
         if (now > lastLog + 5000) {
           lastLog = now;
-          LOG.info("total=" + tot + ", unassigned=" + unassigned + ", tasks=" + tasks);
+          LOG.info("total tasks = " + tot + " unassigned = " + unassigned + " tasks=" + tasks);
         }
       }
       if (resubmitted > 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 4a2c942..7582d42 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,9 +313,8 @@ public class TableNamespaceManager {
   }
 
   private boolean isTableAssigned() {
-    // TODO: we have a better way now (wait on event)
-    return masterServices.getAssignmentManager()
-        .getRegionStates().hasTableRegionStates(TableName.NAMESPACE_TABLE_NAME);
+    return !masterServices.getAssignmentManager()
+        .getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
   }
 
   public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index dfc4321..96ea036 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -183,9 +183,8 @@ public class TableStateManager {
 
   @Nullable
   protected TableState readMetaState(TableName tableName) throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
+    if (tableName.equals(TableName.META_TABLE_NAME))
       return new TableState(tableName, TableState.State.ENABLED);
-    }
     return MetaTableAccessor.getTableState(master.getConnection(), tableName);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..ccff6f0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java
@@ -0,0 +1,47 @@
+/**
+ *
+ * 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/a3c5a744/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
deleted file mode 100644
index 42ece16..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignProcedure.java
+++ /dev/null
@@ -1,338 +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.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.RegionState.State;
-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.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. Should we ignore rollback calls to Assign/Unassign then? Or just
- * remove rollback here?
- */
-@InterfaceAudience.Private
-public class AssignProcedure extends RegionTransitionProcedure {
-  private static final Log LOG = LogFactory.getLog(AssignProcedure.class);
-
-  private boolean forceNewPlan = false;
-
-  /**
-   * Gets set as desired target on move, merge, etc., when we want to go to a particular server.
-   * We may not be able to respect this request but will try. When it is NOT set, then we ask
-   * the balancer to assign. This value is used below in startTransition to set regionLocation if
-   * non-null. Setting regionLocation in regionServerNode is how we override balancer setting
-   * destination.
-   */
-  protected volatile ServerName targetServer;
-
-  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 destinationServer) {
-    super(regionInfo);
-    this.forceNewPlan = false;
-    this.targetServer = destinationServer;
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.REGION_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 (this.targetServer != null) {
-      state.setTargetServer(ProtobufUtil.toServerName(this.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()) {
-      this.targetServer = ProtobufUtil.toServerName(state.getTargetServer());
-    }
-  }
-
-  @Override
-  protected boolean startTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
-      throws IOException {
-    // If the region is already open we can't do much...
-    if (regionNode.isInState(State.OPEN) && isServerOnline(env, regionNode)) {
-      LOG.info("Assigned, not reassigning; " + this + "; " + regionNode.toShortString());
-      return false;
-    }
-    // If the region is SPLIT, we can't assign it.
-    if (regionNode.isInState(State.SPLIT)) {
-      LOG.info("SPLIT, cannot be assigned; " + this + "; " + regionNode.toShortString());
-      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. Setting offline state
-    // scrubs what was the old region location. Setting a new regionLocation here is how we retain
-    // old assignment or specify target server if a move or merge. See
-    // AssignmentManager#processAssignQueue. Otherwise, balancer gives us location.
-    ServerName lastRegionLocation = regionNode.offline();
-    boolean retain = false;
-    if (!forceNewPlan) {
-      if (this.targetServer != null) {
-        retain = targetServer.equals(lastRegionLocation);
-        regionNode.setRegionLocation(targetServer);
-      } else {
-        if (lastRegionLocation != null) {
-          // Try and keep the location we had before we offlined.
-          retain = true;
-          regionNode.setRegionLocation(lastRegionLocation);
-        }
-      }
-    }
-    LOG.info("Start " + this + "; " + regionNode.toShortString() +
-        "; forceNewPlan=" + this.forceNewPlan +
-        ", retain=" + retain);
-    env.getAssignmentManager().queueAssign(regionNode);
-    return true;
-  }
-
-  @Override
-  protected boolean updateTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
-  throws IOException, ProcedureSuspendedException {
-    // TODO: crash if destinationServer is specified and not online
-    // which is also the case when the balancer provided us with a different location.
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Update " + this + "; " + regionNode.toShortString());
-    }
-    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, re-queuing " + this + "; " + regionNode.toShortString());
-      setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
-      return true;
-    }
-
-    if (env.getAssignmentManager().waitServerReportEvent(regionNode.getRegionLocation(), this)) {
-      LOG.info("Early suspend! " + this + "; " + regionNode.toShortString());
-      throw new ProcedureSuspendedException();
-    }
-
-    if (regionNode.isInState(State.OPEN)) {
-      LOG.info("Already assigned: " + this + "; " + regionNode.toShortString());
-      return false;
-    }
-
-    // Transition regionNode State. Set it to OPENING. Update hbase:meta, and add
-    // region to list of regions on the target regionserver. Need to UNDO if failure!
-    env.getAssignmentManager().markRegionAsOpening(regionNode);
-
-    // TODO: Requires a migration to be open by the RS?
-    // regionNode.getFormatVersion()
-
-    if (!addToRemoteDispatcher(env, regionNode.getRegionLocation())) {
-      // Failed the dispatch BUT addToRemoteDispatcher internally does
-      // cleanup on failure -- even the undoing of markRegionAsOpening above --
-      // so nothing more to do here; in fact we need to get out of here
-      // fast since we've been put back on the scheduler.
-    }
-
-    // We always return true, even if we fail dispatch because addToRemoteDispatcher
-    // failure processing sets state back to REGION_TRANSITION_QUEUE so we try again;
-    // i.e. return true to keep the Procedure running; it has been reset to startover.
-    return true;
-  }
-
-  @Override
-  protected void finishTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
-      throws IOException {
-    env.getAssignmentManager().markRegionAsOpened(regionNode);
-    // This success may have been after we failed open a few times. Be sure to cleanup any
-    // failed open references. See #incrementAndCheckMaxAttempts and where it is called.
-    env.getAssignmentManager().getRegionStates().removeFromFailedOpen(regionNode.getRegionInfo());
-  }
-
-  @Override
-  protected void reportTransition(final MasterProcedureEnv env, final RegionStateNode regionNode,
-      final TransitionCode code, final long openSeqNum) throws UnexpectedStateException {
-    switch (code) {
-      case OPENED:
-        if (openSeqNum < 0) {
-          throw new UnexpectedStateException("Received report unexpected " + code +
-              " transition openSeqNum=" + openSeqNum + ", " + regionNode);
-        }
-        if (openSeqNum < regionNode.getOpenSeqNum()) {
-          LOG.warn("Skipping update of open seqnum with " + openSeqNum +
-              " because current seqnum=" + regionNode.getOpenSeqNum());
-        }
-        regionNode.setOpenSeqNum(openSeqNum);
-        // Leave the state here as OPENING for now. We set it to OPEN in
-        // REGION_TRANSITION_FINISH section where we do a bunch of checks.
-        // regionNode.setState(RegionState.State.OPEN, RegionState.State.OPENING);
-        setTransitionState(RegionTransitionState.REGION_TRANSITION_FINISH);
-        break;
-      case FAILED_OPEN:
-        handleFailure(env, regionNode);
-        break;
-      default:
-        throw new UnexpectedStateException("Received report unexpected " + code +
-            " transition openSeqNum=" + openSeqNum + ", " + regionNode.toShortString() +
-            ", " + this + ", expected OPENED or FAILED_OPEN.");
-    }
-  }
-
-  /**
-   * Called when dispatch or subsequent OPEN request fail. Can be run by the
-   * inline dispatch call or later by the ServerCrashProcedure. Our state is
-   * generally OPENING. Cleanup and reset to OFFLINE and put our Procedure
-   * State back to REGION_TRANSITION_QUEUE so the Assign starts over.
-   */
-  private void handleFailure(final MasterProcedureEnv env, final RegionStateNode regionNode) {
-    if (incrementAndCheckMaxAttempts(env, regionNode)) {
-      aborted.set(true);
-    }
-    this.forceNewPlan = true;
-    this.targetServer = null;
-    regionNode.offline();
-    // We were moved to OPENING state before dispatch. Undo. It is safe to call
-    // this method because it checks for OPENING first.
-    env.getAssignmentManager().undoRegionAsOpening(regionNode);
-    setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
-  }
-
-  private boolean incrementAndCheckMaxAttempts(final MasterProcedureEnv env,
-      final RegionStateNode regionNode) {
-    final int retries = env.getAssignmentManager().getRegionStates().
-        addToFailedOpen(regionNode).incrementAndGetRetries();
-    int max = env.getAssignmentManager().getAssignMaxAttempts();
-    LOG.info("Retry=" + retries + " of max=" + max + "; " +
-        this + "; " + regionNode.toShortString());
-    return retries >= max;
-  }
-
-  @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) {
-    handleFailure(env, regionNode);
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    super.toStringClassDetails(sb);
-    if (this.targetServer != null) sb.append(", target=").append(this.targetServer);
-  }
-
-  @Override
-  public ServerName getServer(final MasterProcedureEnv env) {
-    RegionStateNode node =
-        env.getAssignmentManager().getRegionStates().getRegionNode(this.getRegionInfo());
-    if (node == null) return null;
-    return node.getRegionLocation();
-  }
-}
\ No newline at end of file


[04/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index d558aaf..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ /dev/null
@@ -1,358 +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.assignment;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.SortedSet;
-
-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.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.ServerLoad;
-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.HConnectionTestingUtility;
-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.ProcedureEvent;
-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;
-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.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * A mocked master services.
- * Tries to fake it. May not always work.
- */
-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 final ClusterConnection connection;
-  private final LoadBalancer balancer;
-  private final ServerManager serverManager;
-  // Set of regions on a 'server'. Populated externally. Used in below faking 'cluster'.
-  private final NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers;
-
-  private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
-  public static final String DEFAULT_COLUMN_FAMILY_NAME = "cf";
-  public static final ServerName MOCK_MASTER_SERVERNAME =
-      ServerName.valueOf("mockmaster.example.org", 1234, -1L);
-
-  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);
-    // Mock an AM.
-    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);
-
-    // Mock up a Client Interface
-    ClientProtos.ClientService.BlockingInterface ri =
-        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
-    MutateResponse.Builder builder = MutateResponse.newBuilder();
-    builder.setProcessed(true);
-    try {
-      Mockito.when(ri.mutate((RpcController)Mockito.any(), (MutateRequest)Mockito.any())).
-        thenReturn(builder.build());
-    } catch (ServiceException se) {
-      throw ProtobufUtil.handleRemoteException(se);
-    }
-    try {
-      Mockito.when(ri.multi((RpcController)Mockito.any(), (MultiRequest)Mockito.any())).
-        thenAnswer(new Answer<MultiResponse>() {
-          @Override
-          public MultiResponse answer(InvocationOnMock invocation) throws Throwable {
-            return buildMultiResponse( (MultiRequest)invocation.getArguments()[1]);
-          }
-        });
-    } catch (ServiceException se) {
-      throw ProtobufUtil.getRemoteException(se);
-    }
-    // Mock n ClusterConnection and an AdminProtocol implementation. Have the
-    // ClusterConnection return the HRI.  Have the HRI return a few mocked up responses
-    // to make our test work.
-    this.connection =
-        HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
-          Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, MOCK_MASTER_SERVERNAME,
-          HRegionInfo.FIRST_META_REGIONINFO);
-    // Set hbase.rootdir into test dir.
-    Path rootdir = FSUtils.getRootDir(getConfiguration());
-    FSUtils.setRootDir(getConfiguration(), rootdir);
-    Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
-  }
-
-  public void start(final int numServes, final RSProcedureDispatcher remoteDispatcher)
-      throws IOException {
-    startProcedureExecutor(remoteDispatcher);
-    this.assignmentManager.start();
-    for (int i = 0; i < numServes; ++i) {
-      serverManager.regionServerReport(
-        ServerName.valueOf("localhost", 100 + i, 1), ServerLoad.EMPTY_SERVERLOAD);
-    }
-    this.procedureExecutor.getEnvironment().setEventReady(initialized, true);
-  }
-
-  @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));
-    this.procedureStore = new NoopProcedureStore();
-    this.procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
-
-    this.procedureEnv = new MasterProcedureEnv(this,
-       remoteDispatcher != null ? remoteDispatcher : new RSProcedureDispatcher(this));
-
-    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);
-    this.procedureStore.start(numThreads);
-    this.procedureExecutor.start(numThreads, abortOnCorruption);
-    this.procedureEnv.getRemoteDispatcher().start();
-  }
-
-  private void stopProcedureExecutor() {
-    if (this.procedureEnv != null) {
-      this.procedureEnv.getRemoteDispatcher().stop();
-    }
-
-    if (this.procedureExecutor != null) {
-      this.procedureExecutor.stop();
-    }
-
-    if (this.procedureStore != null) {
-      this.procedureStore.stop(isAborted());
-    }
-  }
-
-  @Override
-  public boolean isInitialized() {
-    return true;
-  }
-
-  @Override
-  public ProcedureEvent getInitializedEvent() {
-    return this.initialized;
-  }
-
-  @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;
-  }
-
-  @Override
-  public ClusterConnection getConnection() {
-    return this.connection;
-  }
-
-  @Override
-  public ServerName getServerName() {
-    return MOCK_MASTER_SERVERNAME;
-  }
-
-  @Override
-  public CoordinatedStateManager getCoordinatedStateManager() {
-    return super.getCoordinatedStateManager();
-  }
-
-  private static class MockRegionStateStore extends RegionStateStore {
-    public MockRegionStateStore(final MasterServices master) {
-      super(master);
-    }
-
-    @Override
-    public void start() throws IOException {
-    }
-
-    @Override
-    public void stop() {
-    }
-
-    @Override
-    public void updateRegionLocation(HRegionInfo regionInfo, State state, ServerName regionLocation,
-        ServerName lastHost, long openSeqNum, long pid) throws IOException {
-    }
-  }
-
-  @Override
-  public TableDescriptors getTableDescriptors() {
-    return new TableDescriptors() {
-      @Override
-      public HTableDescriptor remove(TableName tablename) throws IOException {
-        // noop
-        return null;
-      }
-
-      @Override
-      public Map<String, HTableDescriptor> getAll() throws IOException {
-        // noop
-        return null;
-      }
-
-      @Override public Map<String, HTableDescriptor> getAllDescriptors() throws IOException {
-        // noop
-        return null;
-      }
-
-      @Override
-      public HTableDescriptor get(TableName tablename) throws IOException {
-        HTableDescriptor htd = new HTableDescriptor(tablename);
-        htd.addFamily(new HColumnDescriptor(DEFAULT_COLUMN_FAMILY_NAME));
-        return htd;
-      }
-
-      @Override
-      public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
-        return null;
-      }
-
-      @Override
-      public void add(HTableDescriptor htd) throws IOException {
-        // noop
-      }
-
-      @Override
-      public void setCacheOn() throws IOException {
-      }
-
-      @Override
-      public void setCacheOff() throws IOException {
-      }
-    };
-  }
-
-  private static MultiResponse buildMultiResponse(MultiRequest req) {
-    MultiResponse.Builder builder = MultiResponse.newBuilder();
-    RegionActionResult.Builder regionActionResultBuilder =
-        RegionActionResult.newBuilder();
-    ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
-    for (RegionAction regionAction: req.getRegionActionList()) {
-      regionActionResultBuilder.clear();
-      for (ClientProtos.Action action: regionAction.getActionList()) {
-        roeBuilder.clear();
-        roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
-        roeBuilder.setIndex(action.getIndex());
-        regionActionResultBuilder.addResultOrException(roeBuilder.build());
-      }
-      builder.addRegionActionResult(regionActionResultBuilder.build());
-    }
-    return builder.build();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index dda41e0..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ /dev/null
@@ -1,750 +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.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.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-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.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RetriesExhaustedException;
-import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-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.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
-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.Ignore;
-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 / 100);
-
-  private HBaseTestingUtility UTIL;
-  private MockRSProcedureDispatcher rsDispatcher;
-  private MockMasterServices master;
-  private AssignmentManager am;
-  private NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers =
-      new ConcurrentSkipListMap<ServerName, SortedSet<byte []>>();
-  // Simple executor to run some simple tasks.
-  private ScheduledExecutorService executor;
-
-  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, 100); // Have many so we succeed eventually.
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    UTIL = new HBaseTestingUtility();
-    this.executor = Executors.newSingleThreadScheduledExecutor();
-    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.assign(HRegionInfo.FIRST_META_REGIONINFO);
-    am.wakeMetaLoadedEvent();
-    am.setFailoverCleanupDone(true);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    master.stop("tearDown");
-    this.executor.shutdownNow();
-  }
-
-  @Test (expected=NullPointerException.class)
-  public void testWaitServerReportEventWithNullServer() throws UnexpectedStateException {
-    // Test what happens if we pass in null server. I'd expect it throws NPE.
-    if (this.am.waitServerReportEvent(null, null)) throw new UnexpectedStateException();
-  }
-
-  @Ignore @Test // TODO
-  public void testGoodSplit() throws Exception {
-    TableName tableName = TableName.valueOf(this.name.getMethodName());
-    HRegionInfo hri = new HRegionInfo(tableName, Bytes.toBytes(0), Bytes.toBytes(2), false, 0);
-    SplitTableRegionProcedure split =
-        new SplitTableRegionProcedure(this.master.getMasterProcedureExecutor().getEnvironment(),
-            hri, Bytes.toBytes(1));
-    rsDispatcher.setMockRsExecutor(new GoodSplitExecutor());
-    long st = System.currentTimeMillis();
-    Thread t = new Thread() {
-      public void run() {
-        try {
-          waitOnFuture(submitProcedure(split));
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    };
-    t.start();
-    t.join();
-    long et = System.currentTimeMillis();
-    float sec = ((et - st) / 1000.0f);
-    LOG.info(String.format("[T] Splitting in %s", StringUtils.humanTimeDiff(et - st)));
-  }
-
-  @Test
-  public void testAssignWithGoodExec() throws Exception {
-    testAssign(new GoodRsExecutor());
-  }
-
-  @Test
-  public void testAssignAndCrashBeforeResponse() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
-    final HRegionInfo hri = createRegionInfo(tableName, 1);
-    rsDispatcher.setMockRsExecutor(new HangThenRSCrashExecutor());
-    AssignProcedure proc = am.createAssignProcedure(hri, false);
-    waitOnFuture(submitProcedure(proc));
-  }
-
-  @Test
-  public void testUnassignAndCrashBeforeResponse() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
-    final HRegionInfo hri = createRegionInfo(tableName, 1);
-    rsDispatcher.setMockRsExecutor(new HangOnCloseThenRSCrashExecutor());
-    for (int i = 0; i < HangOnCloseThenRSCrashExecutor.TYPES_OF_FAILURE; i++) {
-      AssignProcedure assign = am.createAssignProcedure(hri, false);
-      waitOnFuture(submitProcedure(assign));
-      UnassignProcedure unassign = am.createUnassignProcedure(hri,
-          am.getRegionStates().getRegionServerOfRegion(hri), false);
-      waitOnFuture(submitProcedure(unassign));
-    }
-  }
-
-  @Test
-  public void testAssignWithRandExec() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignWithRandExec");
-    final HRegionInfo hri = createRegionInfo(tableName, 1);
-
-    rsDispatcher.setMockRsExecutor(new RandRsExecutor());
-    // Loop a bunch of times so we hit various combos of exceptions.
-    for (int i = 0; i < 10; i++) {
-      LOG.info("" + i);
-      AssignProcedure proc = am.createAssignProcedure(hri, false);
-      waitOnFuture(submitProcedure(proc));
-    }
-  }
-
-  @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(this.name.getMethodName()),
-      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)));
-
-    // TODO: Currently unassign just keeps trying until it sees a server crash.
-    // There is no count on unassign.
-    /*
-    // 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,
-            ServerName.valueOf("example.org,1234,1"), 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(5, TimeUnit.SECONDS);
-    } catch (ExecutionException e) {
-      LOG.info("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 void doCrash(final ServerName serverName) {
-    this.am.submitServerCrash(serverName, false/*No WALs here*/);
-  }
-
-  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);
-      }
-    }
-  }
-
-  /**
-   * Takes open request and then returns nothing so acts like a RS that went zombie.
-   * No response (so proc is stuck/suspended on the Master and won't wake up.). We
-   * then send in a crash for this server after a few seconds; crash is supposed to
-   * take care of the suspended procedures.
-   */
-  private class HangThenRSCrashExecutor extends GoodRsExecutor {
-    private int invocations;
-
-    @Override
-    protected RegionOpeningState execOpenRegion(final ServerName server, RegionOpenInfo openReq)
-    throws IOException {
-      if (this.invocations++ > 0) {
-        // Return w/o problem the second time through here.
-        return super.execOpenRegion(server, openReq);
-      }
-      // The procedure on master will just hang forever because nothing comes back
-      // from the RS in this case.
-      LOG.info("Return null response from serverName=" + server + "; means STUCK...TODO timeout");
-      executor.schedule(new Runnable() {
-        @Override
-        public void run() {
-          LOG.info("Sending in CRASH of " + server);
-          doCrash(server);
-        }
-      }, 1, TimeUnit.SECONDS);
-      return null;
-    }
-  }
-
-  private class HangOnCloseThenRSCrashExecutor extends GoodRsExecutor {
-    public static final int TYPES_OF_FAILURE = 6;
-    private int invocations;
-
-    @Override
-    protected CloseRegionResponse execCloseRegion(ServerName server, byte[] regionName)
-        throws IOException {
-      switch (this.invocations++) {
-      case 0: throw new NotServingRegionException("Fake");
-      case 1: throw new RegionServerAbortedException("Fake!");
-      case 2: throw new RegionServerStoppedException("Fake!");
-      case 3: throw new ServerNotRunningYetException("Fake!");
-      case 4:
-        LOG.info("Return null response from serverName=" + server + "; means STUCK...TODO timeout");
-        executor.schedule(new Runnable() {
-          @Override
-          public void run() {
-            LOG.info("Sending in CRASH of " + server);
-            doCrash(server);
-          }
-        }, 1, TimeUnit.SECONDS);
-        return null;
-      default:
-        return super.execCloseRegion(server, regionName);
-      }
-    }
-  }
-
-  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(final ServerName server, RegionOpenInfo openReq)
-        throws IOException {
-      switch (rand.nextInt(6)) {
-        case 0:
-          LOG.info("Return OPENED response");
-          sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED);
-          return OpenRegionResponse.RegionOpeningState.OPENED;
-        case 1:
-          LOG.info("Return transition report that OPENED/ALREADY_OPENED response");
-          sendTransitionReport(server, openReq.getRegion(), TransitionCode.OPENED);
-          return OpenRegionResponse.RegionOpeningState.ALREADY_OPENED;
-        case 2:
-          LOG.info("Return transition report that FAILED_OPEN/FAILED_OPENING response");
-          sendTransitionReport(server, openReq.getRegion(), TransitionCode.FAILED_OPEN);
-          return OpenRegionResponse.RegionOpeningState.FAILED_OPENING;
-      }
-      // The procedure on master will just hang forever because nothing comes back
-      // from the RS in this case.
-      LOG.info("Return null as response; means proc stuck so we send in a crash report after a few seconds...");
-      executor.schedule(new Runnable() {
-        @Override
-        public void run() {
-          LOG.info("Delayed CRASHING of " + server);
-          doCrash(server);
-        }
-      }, 5, TimeUnit.SECONDS);
-      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);
-      }
-    }
-  }
-
-  private class GoodSplitExecutor 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();
-    }*/
-    
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index e4cec45..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
+++ /dev/null
@@ -1,185 +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.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/a3c5a744/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
deleted file mode 100644
index 44fd575..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMergeTableRegionsProcedure.java
+++ /dev/null
@@ -1,260 +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.assignment;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-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.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.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-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.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-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 TestMergeTableRegionsProcedure {
-  private static final Log LOG = LogFactory.getLog(TestMergeTableRegionsProcedure.class);
-  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
-      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
-  @Rule public final TestName name = new TestName();
-
-  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
-  public void testMergeTwoRegions() throws Exception {
-    final TableName tableName = TableName.valueOf(this.name.getMethodName());
-    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);
-    MergeTableRegionsProcedure proc =
-        new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true);
-    long procId = procExec.submitProcedure(proc);
-    ProcedureTestingUtility.waitProcedure(procExec, procId);
-    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-    assertRegionCount(tableName, initialRegionCount - 1);
-    Pair<HRegionInfo, HRegionInfo> pair =
-      MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
-        proc.getMergedRegion().getRegionName());
-    assertTrue(pair.getFirst() != null && pair.getSecond() != null);
-
-    // Can I purge the merged regions from hbase:meta? Check that all went
-    // well by looking at the merged row up in hbase:meta. It should have no
-    // more mention of the merged regions; they are purged as last step in
-    // the merged regions cleanup.
-    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
-    UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
-    while (pair != null && pair.getFirst() != null && pair.getSecond() != null) {
-      pair = MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
-          proc.getMergedRegion().getRegionName());
-    }
-  }
-
-  /**
-   * This tests two concurrent region merges
-   */
-  @Test
-  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
-  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
-  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/a3c5a744/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
deleted file mode 100644
index 003dfdd..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStates.java
+++ /dev/null
@@ -1,224 +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.assignment;
-
-import static org.junit.Assert.assertEquals;
-
-import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.List;
-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 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.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;
-
-@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);
-    }
-    waitExecutorService(NSMALL_RUNS * 2);
-    // 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, true);
-    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
-  // ==========================================================================
-}


[06/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..23e61f6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -0,0 +1,1403 @@
+/**
+ * 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.HBaseIOException;
+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) throws HBaseIOException {
+      if (regionInfo.equals(controledRegion)) {
+        return null;
+      }
+      return super.randomAssignment(regionInfo, servers);
+    }
+
+    @Override
+    public Map<ServerName, List<HRegionInfo>> roundRobinAssignment(
+        List<HRegionInfo> regions, List<ServerName> servers) throws HBaseIOException {
+      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) throws HBaseIOException {
+      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();
+      }
+    }
+  }
+}


[26/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 b648cf2..1bb6118 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
@@ -32,8 +32,6 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.DelayQueue;
@@ -115,11 +113,9 @@ public class ProcedureExecutor<TEnvironment> {
    * Internal cleaner that removes the completed procedure results after a TTL.
    * NOTE: This is a special case handled in timeoutLoop().
    *
-   * <p>Since the client code looks more or less like:
-   * <pre>
+   * Since the client code looks more or less like:
    *   procId = master.doOperation()
    *   while (master.getProcResult(procId) == ProcInProgress);
-   * </pre>
    * The master should not throw away the proc result as soon as the procedure is done
    * but should wait a result request from the client (see executor.removeResult(procId))
    * The client will call something like master.isProcDone() or master.getProcResult()
@@ -484,10 +480,10 @@ 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 ProcedureExecutor Worker threads (ProcExecWrkr)=" + corePoolSize);
+    LOG.info("Starting executor worker threads=" + corePoolSize);
 
     // Create the Thread Group for the executors
-    threadGroup = new ThreadGroup("ProcExecThrdGrp");
+    threadGroup = new ThreadGroup("ProcedureExecutor");
 
     // Create the timeout executor
     timeoutExecutor = new TimeoutExecutorThread(threadGroup);
@@ -1081,16 +1077,13 @@ public class ProcedureExecutor<TEnvironment> {
     final Long rootProcId = getRootProcedureId(proc);
     if (rootProcId == null) {
       // The 'proc' was ready to run but the root procedure was rolledback
-      LOG.warn("Rollback because parent is done/rolledback proc=" + proc);
       executeRollback(proc);
       return;
     }
 
     final RootProcedureState procStack = rollbackStack.get(rootProcId);
-    if (procStack == null) {
-      LOG.warn("RootProcedureState is null for " + proc.getProcId());
-      return;
-    }
+    if (procStack == null) return;
+
     do {
       // Try to acquire the execution
       if (!procStack.acquire(proc)) {
@@ -1104,7 +1097,6 @@ public class ProcedureExecutor<TEnvironment> {
               scheduler.yield(proc);
               break;
             case LOCK_EVENT_WAIT:
-              LOG.info("LOCK_EVENT_WAIT rollback..." + proc);
               procStack.unsetRollback();
               break;
             default:
@@ -1122,7 +1114,6 @@ public class ProcedureExecutor<TEnvironment> {
                 scheduler.yield(proc);
                 break;
               case LOCK_EVENT_WAIT:
-                LOG.info("LOCK_EVENT_WAIT can't rollback child running?..." + proc);
                 break;
               default:
                 throw new UnsupportedOperationException();
@@ -1134,21 +1125,16 @@ public class ProcedureExecutor<TEnvironment> {
 
       // Execute the procedure
       assert proc.getState() == ProcedureState.RUNNABLE : proc;
-      // Note that lock is NOT about concurrency but rather about ensuring
-      // ownership of a procedure of an entity such as a region or table
-      LockState lockState = acquireLock(proc);
-      switch (lockState) {
+      switch (acquireLock(proc)) {
         case LOCK_ACQUIRED:
           execProcedure(procStack, proc);
           releaseLock(proc, false);
           break;
         case LOCK_YIELD_WAIT:
-          LOG.info(lockState + " " + proc);
           scheduler.yield(proc);
           break;
         case LOCK_EVENT_WAIT:
-          // Someone will wake us up when the lock is available
-          LOG.debug(lockState + " " + proc);
+          // someone will wake us up when the lock is available
           break;
         default:
           throw new UnsupportedOperationException();
@@ -1164,7 +1150,10 @@ public class ProcedureExecutor<TEnvironment> {
       if (proc.isSuccess()) {
         // update metrics on finishing the procedure
         proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true);
-        LOG.info("Finish " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Finished " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime()));
+        }
         // Finalize the procedure state
         if (proc.getProcId() == rootProcId) {
           procedureFinished(proc);
@@ -1189,7 +1178,7 @@ public class ProcedureExecutor<TEnvironment> {
 
   private void releaseLock(final Procedure proc, final boolean force) {
     final TEnvironment env = getEnvironment();
-    // For how the framework works, we know that we will always have the lock
+    // for how the framework works, we know that we will always have the lock
     // when we call releaseLock(), so we can avoid calling proc.hasLock()
     if (force || !proc.holdLock(env)) {
       proc.doReleaseLock(env);
@@ -1204,8 +1193,6 @@ public class ProcedureExecutor<TEnvironment> {
   private LockState executeRollback(final long rootProcId, final RootProcedureState procStack) {
     final Procedure rootProc = procedures.get(rootProcId);
     RemoteProcedureException exception = rootProc.getException();
-    // TODO: This needs doc. The root proc doesn't have an exception. Maybe we are
-    // rolling back because the subprocedure does. Clarify.
     if (exception == null) {
       exception = procStack.getException();
       rootProc.setFailure(exception);
@@ -1282,7 +1269,7 @@ public class ProcedureExecutor<TEnvironment> {
       return LockState.LOCK_YIELD_WAIT;
     } catch (Throwable e) {
       // Catch NullPointerExceptions or similar errors...
-      LOG.fatal("CODE-BUG: Uncaught runtime exception fo " + proc, e);
+      LOG.fatal("CODE-BUG: Uncatched runtime exception for procedure: " + proc, e);
     }
 
     // allows to kill the executor before something is stored to the wal.
@@ -1318,55 +1305,29 @@ public class ProcedureExecutor<TEnvironment> {
   }
 
   /**
-   * Executes <code>procedure</code>
-   * <ul>
-   *  <li>Calls the doExecute() of the procedure
-   *  <li>If the procedure execution didn't fail (i.e. valid user input)
-   *  <ul>
-   *    <li>...and returned subprocedures
-   *    <ul><li>The subprocedures are initialized.
-   *      <li>The subprocedures are added to the store
-   *      <li>The subprocedures are added to the runnable queue
-   *      <li>The procedure is now in a WAITING state, waiting for the subprocedures to complete
-   *    </ul>
-   *    </li>
-   *   <li>...if there are no subprocedure
-   *    <ul><li>the procedure completed successfully
-   *      <li>if there is a parent (WAITING)
-   *      <li>the parent state will be set to RUNNABLE
-   *    </ul>
-   *   </li>
-   *  </ul>
-   *  </li>
-   *  <li>In case of failure
-   *  <ul>
-   *    <li>The store is updated with the new state</li>
-   *    <li>The executor (caller of this method) will start the rollback of the procedure</li>
-   *  </ul>
-   *  </li>
-   *  </ul>
+   * Executes the specified procedure
+   *  - calls the doExecute() of the procedure
+   *  - if the procedure execution didn't fail (e.g. invalid user input)
+   *     - ...and returned subprocedures
+   *        - the subprocedures are initialized.
+   *        - the subprocedures are added to the store
+   *        - the subprocedures are added to the runnable queue
+   *        - the procedure is now in a WAITING state, waiting for the subprocedures to complete
+   *     - ...if there are no subprocedure
+   *        - the procedure completed successfully
+   *        - if there is a parent (WAITING)
+   *            - the parent state will be set to RUNNABLE
+   *  - in case of failure
+   *    - the store is updated with the new state
+   *    - the executor (caller of this method) will start the rollback of the procedure
    */
-  private void execProcedure(final RootProcedureState procStack,
-      final Procedure<TEnvironment> procedure) {
+  private void execProcedure(final RootProcedureState procStack, final Procedure procedure) {
     Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
 
-    // Procedures can suspend themselves. They skip out by throwing a ProcedureSuspendedException.
-    // The exception is caught below and then we hurry to the exit without disturbing state. The
-    // idea is that the processing of this procedure will be unsuspended later by an external event
-    // such the report of a region open. TODO: Currently, its possible for two worker threads
-    // to be working on the same procedure concurrently (locking in procedures is NOT about
-    // concurrency but about tying an entity to a procedure; i.e. a region to a particular
-    // procedure instance). This can make for issues if both threads are changing state.
-    // See env.getProcedureScheduler().wakeEvent(regionNode.getProcedureEvent());
-    // in RegionTransitionProcedure#reportTransition for example of Procedure putting
-    // itself back on the scheduler making it possible for two threads running against
-    // the one Procedure. Might be ok if they are both doing different, idempotent sections.
+    // Execute the procedure
     boolean suspended = false;
-
-    // Whether to 're-' -execute; run through the loop again.
     boolean reExecute = false;
-
-    Procedure<TEnvironment>[] subprocs = null;
+    Procedure[] subprocs = null;
     do {
       reExecute = false;
       try {
@@ -1375,20 +1336,14 @@ public class ProcedureExecutor<TEnvironment> {
           subprocs = null;
         }
       } catch (ProcedureSuspendedException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Suspend " + procedure);
-        }
         suspended = true;
       } catch (ProcedureYieldException e) {
         if (LOG.isTraceEnabled()) {
-          LOG.trace("Yield " + procedure + ": " + e.getMessage(), e);
+          LOG.trace("Yield " + procedure + ": " + e.getMessage());
         }
         scheduler.yield(procedure);
         return;
       } catch (InterruptedException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Yield interrupt " + procedure + ": " + e.getMessage(), e);
-        }
         handleInterruptedException(procedure, e);
         scheduler.yield(procedure);
         return;
@@ -1402,26 +1357,14 @@ public class ProcedureExecutor<TEnvironment> {
       if (!procedure.isFailed()) {
         if (subprocs != null) {
           if (subprocs.length == 1 && subprocs[0] == procedure) {
-            // Procedure returned itself. Quick-shortcut for a state machine-like procedure;
-            // i.e. we go around this loop again rather than go back out on the scheduler queue.
+            // quick-shortcut for a state machine like procedure
             subprocs = null;
             reExecute = true;
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Short-circuit to next step on pid=" + procedure.getProcId());
-            }
           } else {
-            // Yield the current procedure, and make the subprocedure runnable
-            // subprocs may come back 'null'.
+            // yield the current procedure, and make the subprocedure runnable
             subprocs = initializeChildren(procStack, procedure, subprocs);
-            LOG.info("Initialized subprocedures=" +
-              (subprocs == null? null:
-                Stream.of(subprocs).map(e -> "{" + e.toString() + "}").
-                collect(Collectors.toList()).toString()));
           }
         } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Added to timeoutExecutor " + procedure);
-          }
           timeoutExecutor.add(procedure);
         } else if (!suspended) {
           // No subtask, so we are done
@@ -1445,13 +1388,12 @@ public class ProcedureExecutor<TEnvironment> {
       // executor thread to stop. The statement following the method call below seems to check if
       // store is not running, to prevent scheduling children procedures, re-execution or yield
       // of this procedure. This may need more scrutiny and subsequent cleanup in future
-      //
-      // Commit the transaction even if a suspend (state may have changed). Note this append
-      // can take a bunch of time to complete.
+      // Commit the transaction
       updateStoreOnExec(procStack, procedure, subprocs);
 
       // if the store is not running we are aborting
       if (!store.isRunning()) return;
+
       // if the procedure is kind enough to pass the slot to someone else, yield
       if (procedure.isRunnable() && !suspended &&
           procedure.isYieldAfterExecutionStep(getEnvironment())) {
@@ -1461,14 +1403,14 @@ public class ProcedureExecutor<TEnvironment> {
 
       assert (reExecute && subprocs == null) || !reExecute;
     } while (reExecute);
+
     // Submit the new subprocedures
     if (subprocs != null && !procedure.isFailed()) {
       submitChildrenProcedures(subprocs);
     }
 
-    // if the procedure is complete and has a parent, count down the children latch.
-    // If 'suspended', do nothing to change state -- let other threads handle unsuspend event.
-    if (!suspended && procedure.isFinished() && procedure.hasParent()) {
+    // if the procedure is complete and has a parent, count down the children latch
+    if (procedure.isFinished() && procedure.hasParent()) {
       countDownChildren(procStack, procedure);
     }
   }
@@ -1527,13 +1469,18 @@ public class ProcedureExecutor<TEnvironment> {
     }
 
     // If this procedure is the last child awake the parent procedure
-    LOG.info("Finish suprocedure " + procedure);
-    if (parent.tryRunnable()) {
-      // If we succeeded in making the parent runnable -- i.e. all of its
-      // children have completed, move parent to front of the queue.
+    final boolean traceEnabled = LOG.isTraceEnabled();
+    if (traceEnabled) {
+      LOG.trace(parent + " child is done: " + procedure);
+    }
+
+    if (parent.childrenCountDown() && parent.getState() == ProcedureState.WAITING) {
+      parent.setState(ProcedureState.RUNNABLE);
       store.update(parent);
       scheduler.addFront(parent);
-      LOG.info("Finished subprocedure(s) of " + parent + "; resume parent processing.");
+      if (traceEnabled) {
+        LOG.trace(parent + " all the children finished their work, resume.");
+      }
       return;
     }
   }
@@ -1622,10 +1569,9 @@ public class ProcedureExecutor<TEnvironment> {
   // ==========================================================================
   private final class WorkerThread extends StoppableThread {
     private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE);
-    private Procedure activeProcedure;
 
     public WorkerThread(final ThreadGroup group) {
-      super(group, "ProcExecWrkr-" + workerId.incrementAndGet());
+      super(group, "ProcExecWorker-" + workerId.incrementAndGet());
     }
 
     @Override
@@ -1635,49 +1581,29 @@ public class ProcedureExecutor<TEnvironment> {
 
     @Override
     public void run() {
+      final boolean traceEnabled = LOG.isTraceEnabled();
       long lastUpdate = EnvironmentEdgeManager.currentTime();
-      try {
-        while (isRunning() && keepAlive(lastUpdate)) {
-          this.activeProcedure = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
-          if (this.activeProcedure == null) continue;
-          int activeCount = activeExecutorCount.incrementAndGet();
-          int runningCount = store.setRunningProcedureCount(activeCount);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Execute pid=" + this.activeProcedure.getProcId() +
-                " runningCount=" + runningCount + ", activeCount=" + activeCount);
-          }
-          executionStartTime.set(EnvironmentEdgeManager.currentTime());
-          try {
-            executeProcedure(this.activeProcedure);
-          } catch (AssertionError e) {
-            LOG.info("ASSERT pid=" + this.activeProcedure.getProcId(), e);
-            throw e;
-          } finally {
-            activeCount = activeExecutorCount.decrementAndGet();
-            runningCount = store.setRunningProcedureCount(activeCount);
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Halt pid=" + this.activeProcedure.getProcId() +
-                  " runningCount=" + runningCount + ", activeCount=" + activeCount);
-            }
-            this.activeProcedure = null;
-            lastUpdate = EnvironmentEdgeManager.currentTime();
-            executionStartTime.set(Long.MAX_VALUE);
+      while (isRunning() && keepAlive(lastUpdate)) {
+        final Procedure procedure = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
+        if (procedure == null) continue;
+
+        store.setRunningProcedureCount(activeExecutorCount.incrementAndGet());
+        executionStartTime.set(EnvironmentEdgeManager.currentTime());
+        try {
+          if (traceEnabled) {
+            LOG.trace("Trying to start the execution of " + procedure);
           }
+          executeProcedure(procedure);
+        } finally {
+          store.setRunningProcedureCount(activeExecutorCount.decrementAndGet());
+          lastUpdate = EnvironmentEdgeManager.currentTime();
+          executionStartTime.set(Long.MAX_VALUE);
         }
-      } catch (Throwable t) {
-        LOG.warn("Worker terminating UNNATURALLY " + this.activeProcedure, t);
-      } finally {
-        LOG.debug("Worker terminated.");
       }
+      LOG.debug("Worker thread terminated " + this);
       workerThreads.remove(this);
     }
 
-    @Override
-    public String toString() {
-      Procedure<?> p = this.activeProcedure;
-      return getName() + "(pid=" + (p == null? Procedure.NO_PROC_ID: p.getProcId() + ")");
-    }
-
     /**
      * @return the time since the current procedure is running
      */
@@ -1691,15 +1617,14 @@ public class ProcedureExecutor<TEnvironment> {
     }
   }
 
-  /**
-   * Runs task on a period such as check for stuck workers.
-   * @see InlineChore
-   */
+  // ==========================================================================
+  //  Timeout Thread
+  // ==========================================================================
   private final class TimeoutExecutorThread extends StoppableThread {
     private final DelayQueue<DelayedWithTimeout> queue = new DelayQueue<>();
 
     public TimeoutExecutorThread(final ThreadGroup group) {
-      super(group, "ProcExecTimeout");
+      super(group, "ProcedureTimeoutExecutor");
     }
 
     @Override
@@ -1709,7 +1634,7 @@ public class ProcedureExecutor<TEnvironment> {
 
     @Override
     public void run() {
-      final boolean traceEnabled = LOG.isTraceEnabled();
+      final boolean isTraceEnabled = LOG.isTraceEnabled();
       while (isRunning()) {
         final DelayedWithTimeout task = DelayedUtil.takeWithoutInterrupt(queue);
         if (task == null || task == DelayedUtil.DELAYED_POISON) {
@@ -1718,8 +1643,8 @@ public class ProcedureExecutor<TEnvironment> {
           continue;
         }
 
-        if (traceEnabled) {
-          LOG.trace("Executing " + task);
+        if (isTraceEnabled) {
+          LOG.trace("Trying to start the execution of " + task);
         }
 
         // execute the task
@@ -1740,8 +1665,6 @@ public class ProcedureExecutor<TEnvironment> {
 
     public void add(final Procedure procedure) {
       assert procedure.getState() == ProcedureState.WAITING_TIMEOUT;
-      LOG.info("ADDED " + procedure + "; timeout=" + procedure.getTimeout() +
-          ", timestamp=" + procedure.getTimeoutTimestamp());
       queue.add(new DelayedProcedure(procedure));
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
index b148dae..bdced10 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureInMemoryChore.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 /**
  * Special procedure used as a chore.
- * Instead of bringing the Chore class in (dependencies reason),
+ * instead of bringing the Chore class in (dependencies reason),
  * we reuse the executor timeout thread for this special case.
  *
  * The assumption is that procedure is used as hook to dispatch other procedures
@@ -43,7 +43,7 @@ public abstract class ProcedureInMemoryChore<TEnvironment> extends Procedure<TEn
   protected abstract void periodicExecute(final TEnvironment env);
 
   @Override
-  protected Procedure<TEnvironment>[] execute(final TEnvironment env) {
+  protected Procedure[] execute(final TEnvironment env) {
     throw new UnsupportedOperationException();
   }
 
@@ -66,4 +66,4 @@ public abstract class ProcedureInMemoryChore<TEnvironment> extends Procedure<TEn
   public void deserializeStateData(final InputStream stream) {
     throw new UnsupportedOperationException();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 233ef57..93d0d5d 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
@@ -93,7 +93,7 @@ public interface ProcedureScheduler {
 
   /**
    * Mark the event as not ready.
-   * Procedures calling waitEvent() will be suspended.
+   * procedures calling waitEvent() will be suspended.
    * @param event the event to mark as suspended/not ready
    */
   void suspendEvent(ProcedureEvent event);
@@ -125,7 +125,6 @@ public interface ProcedureScheduler {
    * List lock queues.
    * @return the locks
    */
-  // TODO: This seems to be the wrong place to hang this method.
   List<LockInfo> listLocks();
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 8d5ff3c..0000000
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ /dev/null
@@ -1,375 +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.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(this.getClass().getSimpleName(),
-          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 remote procedures
-   * @param key the node identifier
-   */
-  public void addNode(final TRemote key) {
-    assert key != null: "Tried to add a node with a null key";
-    final BufferNode newNode = new BufferNode(key);
-    nodeMap.putIfAbsent(key, newNode);
-  }
-
-  /**
-   * Add a remote rpc. Be sure to check result for successful add.
-   * @param key the node identifier
-   * @return True if we successfully added the operation.
-   */
-  public boolean addOperationToNode(final TRemote key, RemoteProcedure rp) {
-    assert key != null : "found null key for node";
-    BufferNode node = nodeMap.get(key);
-    if (node == null) {
-      return false;
-    }
-    node.add(rp);
-    // Check our node still in the map; could have been removed by #removeNode.
-    return nodeMap.contains(node);
-  }
-
-  /**
-   * 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/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
index 64bb278..1a84070 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
@@ -27,13 +27,12 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData;
 
 /**
- * A SequentialProcedure describes one step in a procedure chain:
- * <pre>
+ * A SequentialProcedure describes one step in a procedure chain.
  *   -&gt; Step 1 -&gt; Step 2 -&gt; Step 3
- * </pre>
+ *
  * The main difference from a base Procedure is that the execute() of a
- * SequentialProcedure will be called only once; there will be no second
- * execute() call once the children are finished. which means once the child
+ * SequentialProcedure will be called only once, there will be no second
+ * execute() call once the child are finished. which means once the child
  * of a SequentialProcedure are completed the SequentialProcedure is completed too.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 becd9b7..0590a93 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
@@ -21,10 +21,9 @@ package org.apache.hadoop.hbase.procedure2;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -35,7 +34,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMa
 /**
  * Procedure described by a series of steps.
  *
- * <p>The procedure implementor must have an enum of 'states', describing
+ * The procedure implementor must have an enum of 'states', describing
  * the various step of the procedure.
  * Once the procedure is running, the procedure-framework will call executeFromState()
  * using the 'state' provided by the user. The first call to executeFromState()
@@ -57,7 +56,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
   private int stateCount = 0;
   private int[] states = null;
 
-  private List<Procedure<TEnvironment>> subProcList = null;
+  private ArrayList<Procedure> subProcList = null;
 
   protected enum Flow {
     HAS_MORE_STATE,
@@ -71,7 +70,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
    *         Flow.HAS_MORE_STATE if there is another step.
    */
   protected abstract Flow executeFromState(TEnvironment env, TState state)
-  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException;
+    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException;
 
   /**
    * called to perform the rollback of the specified state
@@ -126,15 +125,12 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
    * Add a child procedure to execute
    * @param subProcedure the child procedure
    */
-  protected void addChildProcedure(Procedure<TEnvironment>... subProcedure) {
-    if (subProcedure == null) return;
-    final int len = subProcedure.length;
-    if (len == 0) return;
+  protected void addChildProcedure(Procedure... subProcedure) {
     if (subProcList == null) {
-      subProcList = new ArrayList<>(len);
+      subProcList = new ArrayList<>(subProcedure.length);
     }
-    for (int i = 0; i < len; ++i) {
-      Procedure<TEnvironment> proc = subProcedure[i];
+    for (int i = 0; i < subProcedure.length; ++i) {
+      Procedure proc = subProcedure[i];
       if (!proc.hasOwner()) proc.setOwner(getOwner());
       subProcList.add(proc);
     }
@@ -142,23 +138,27 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
 
   @Override
   protected Procedure[] execute(final TEnvironment env)
-  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     updateTimestamp();
     try {
       failIfAborted();
 
       if (!hasMoreState() || isFailed()) return null;
+
       TState state = getCurrentState();
       if (stateCount == 0) {
         setNextState(getStateId(state));
       }
+
       stateFlow = executeFromState(env, state);
       if (!hasMoreState()) setNextState(EOF_STATE);
-      if (subProcList != null && !subProcList.isEmpty()) {
+
+      if (subProcList != null && subProcList.size() != 0) {
         Procedure[] subProcedures = subProcList.toArray(new Procedure[subProcList.size()]);
         subProcList = null;
         return subProcedures;
       }
+
       return (isWaiting() || isFailed() || !hasMoreState()) ? null : new Procedure[] {this};
     } finally {
       updateTimestamp();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/NoopProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/NoopProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/NoopProcedureStore.java
index 9e53f42..c03e326 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/NoopProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/NoopProcedureStore.java
@@ -52,8 +52,8 @@ public class NoopProcedureStore extends ProcedureStoreBase {
   }
 
   @Override
-  public int setRunningProcedureCount(final int count) {
-    return count;
+  public void setRunningProcedureCount(final int count) {
+    // no-op
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
index a690c81..385cedb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
@@ -153,9 +153,8 @@ public interface ProcedureStore {
   /**
    * Set the number of procedure running.
    * This can be used, for example, by the store to know how long to wait before a sync.
-   * @return how many procedures are running (may not be same as <code>count</code>).
    */
-  int setRunningProcedureCount(int count);
+  void setRunningProcedureCount(int count);
 
   /**
    * Acquire the lease for the procedure store.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 95a1ef6..012ddeb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -155,23 +155,9 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
     this.logSize += size;
   }
 
-  public void removeFile(final Path walArchiveDir) throws IOException {
+  public void removeFile() throws IOException {
     close();
-    boolean archived = false;
-    if (walArchiveDir != null) {
-      Path archivedFile = new Path(walArchiveDir, logFile.getName());
-      LOG.info("ARCHIVED (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + logFile + " to " + archivedFile);
-      if (!fs.rename(logFile, archivedFile)) {
-        LOG.warn("Failed archive of " + logFile + ", deleting");
-      } else {
-        archived = true;
-      }
-    }
-    if (!archived) {
-      if (!fs.delete(logFile, false)) {
-        LOG.warn("Failed delete of " + logFile);
-      }
-    }
+    fs.delete(logFile, false);
   }
 
   public void setProcIds(long minId, long maxId) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
index 0a05e6e..c672045 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
@@ -83,11 +83,11 @@ public class ProcedureWALFormatReader {
   //
   //  Fast Start: INIT/INSERT record and StackIDs
   // ---------------------------------------------
-  // We have two special records, INIT and INSERT, that track the first time
-  // the procedure was added to the WAL. We can use this information to be able
-  // to start procedures before reaching the end of the WAL, or before reading all WALs.
-  // But in some cases, the WAL with that record can be already gone.
-  // As an alternative, we can use the stackIds on each procedure,
+  // We have two special record, INIT and INSERT that tracks the first time
+  // the procedure was added to the WAL. We can use that information to be able
+  // to start procedures before reaching the end of the WAL, or before reading all the WALs.
+  // but in some cases the WAL with that record can be already gone.
+  // In alternative we can use the stackIds on each procedure,
   // to identify when a procedure is ready to start.
   // If there are gaps in the sum of the stackIds we need to read more WALs.
   //
@@ -107,16 +107,16 @@ public class ProcedureWALFormatReader {
    * Global tracker that will be used by the WALProcedureStore after load.
    * If the last WAL was closed cleanly we already have a full tracker ready to be used.
    * If the last WAL was truncated (e.g. master killed) the tracker will be empty
-   * and the 'partial' flag will be set. In this case, on WAL replay we are going
+   * and the 'partial' flag will be set. In this case on WAL replay we are going
    * to rebuild the tracker.
    */
   private final ProcedureStoreTracker tracker;
-  // TODO: private final boolean hasFastStartSupport;
+  // private final boolean hasFastStartSupport;
 
   /**
    * If tracker for a log file is partial (see {@link ProcedureStoreTracker#partial}), we
    * re-build the list of procedures updated in that WAL because we need it for log cleaning
-   * purposes. If all procedures updated in a WAL are found to be obsolete, it can be safely deleted.
+   * purpose. If all procedures updated in a WAL are found to be obsolete, it can be safely deleted.
    * (see {@link WALProcedureStore#removeInactiveLogs()}).
    * However, we don't need deleted part of a WAL's tracker for this purpose, so we don't bother
    * re-building it.
@@ -137,7 +137,7 @@ public class ProcedureWALFormatReader {
   public void read(final ProcedureWALFile log) throws IOException {
     localTracker = log.getTracker().isPartial() ? log.getTracker() : null;
     if (localTracker != null) {
-      LOG.info("Rebuilding tracker for " + log);
+      LOG.info("Rebuilding tracker for log - " + log);
     }
 
     FSDataInputStream stream = log.getStream();
@@ -146,7 +146,7 @@ public class ProcedureWALFormatReader {
       while (hasMore) {
         ProcedureWALEntry entry = ProcedureWALFormat.readEntry(stream);
         if (entry == null) {
-          LOG.warn("Nothing left to decode. Exiting with missing EOF, log=" + log);
+          LOG.warn("nothing left to decode. exiting with missing EOF");
           break;
         }
         switch (entry.getType()) {
@@ -171,7 +171,7 @@ public class ProcedureWALFormatReader {
         }
       }
     } catch (InvalidProtocolBufferException e) {
-      LOG.error("While reading procedure from " + log, e);
+      LOG.error("got an exception while reading the procedure WAL: " + log, e);
       loader.markCorruptedWAL(log, e);
     }
 
@@ -211,7 +211,7 @@ public class ProcedureWALFormatReader {
     maxProcId = Math.max(maxProcId, proc.getProcId());
     if (isRequired(proc.getProcId())) {
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Read " + entry.getType() + " entry " + proc.getProcId());
+        LOG.trace("read " + entry.getType() + " entry " + proc.getProcId());
       }
       localProcedureMap.add(proc);
       if (tracker.isPartial()) {
@@ -296,7 +296,7 @@ public class ProcedureWALFormatReader {
   //      replayOrderHead = C <-> B <-> E <-> D <-> A <-> G
   //
   //  We also have a lazy grouping by "root procedure", and a list of
-  //  unlinked procedures. If after reading all the WALs we have unlinked
+  //  unlinked procedure. If after reading all the WALs we have unlinked
   //  procedures it means that we had a missing WAL or a corruption.
   //      rootHead = A <-> D <-> G
   //                 B     E
@@ -639,17 +639,17 @@ public class ProcedureWALFormatReader {
      * "ready" means that we all the information that we need in-memory.
      *
      * Example-1:
-     * We have two WALs, we start reading from the newest (wal-2)
+     * We have two WALs, we start reading fronm the newest (wal-2)
      *    wal-2 | C B |
      *    wal-1 | A B C |
      *
      * If C and B don't depend on A (A is not the parent), we can start them
-     * before reading wal-1. If B is the only one with parent A we can start C.
-     * We have to read one more WAL before being able to start B.
+     * before reading wal-1. If B is the only one with parent A we can start C
+     * and read one more WAL before being able to start B.
      *
      * How do we know with the only information in B that we are not ready.
      *  - easy case, the parent is missing from the global map
-     *  - more complex case we look at the Stack IDs.
+     *  - more complex case we look at the Stack IDs
      *
      * The Stack-IDs are added to the procedure order as incremental index
      * tracking how many times that procedure was executed, which is equivalent
@@ -664,7 +664,7 @@ public class ProcedureWALFormatReader {
      * executed before.
      * To identify when a Procedure is ready we do the sum of the stackIds of
      * the procedure and the parent. if the stackIdSum is equals to the
-     * sum of {1..maxStackId} then everything we need is available.
+     * sum of {1..maxStackId} then everything we need is avaiable.
      *
      * Example-2
      *    wal-2 | A |              A stackIds = [0, 2]
@@ -676,7 +676,7 @@ public class ProcedureWALFormatReader {
       assert !rootEntry.hasParent() : "expected root procedure, got " + rootEntry;
 
       if (rootEntry.isFinished()) {
-        // If the root procedure is finished, sub-procedures should be gone
+        // if the root procedure is finished, sub-procedures should be gone
         if (rootEntry.childHead != null) {
           LOG.error("unexpected active children for root-procedure: " + rootEntry);
           for (Entry p = rootEntry.childHead; p != null; p = p.linkNext) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 1791cae..4712c30 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -66,7 +66,6 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceStability.Evolving
 public class WALProcedureStore extends ProcedureStoreBase {
   private static final Log LOG = LogFactory.getLog(WALProcedureStore.class);
-  public static final String LOG_PREFIX = "pv2-";
 
   public interface LeaseRecovery {
     void recoverFileLease(FileSystem fs, Path path) throws IOException;
@@ -125,7 +124,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private final Configuration conf;
   private final FileSystem fs;
   private final Path walDir;
-  private final Path walArchiveDir;
 
   private final AtomicReference<Throwable> syncException = new AtomicReference<>();
   private final AtomicBoolean loading = new AtomicBoolean(true);
@@ -187,15 +185,9 @@ public class WALProcedureStore extends ProcedureStoreBase {
 
   public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path walDir,
       final LeaseRecovery leaseRecovery) {
-    this(conf, fs, walDir, null, leaseRecovery);
-  }
-
-  public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path walDir,
-      final Path walArchiveDir, final LeaseRecovery leaseRecovery) {
     this.fs = fs;
     this.conf = conf;
     this.walDir = walDir;
-    this.walArchiveDir = walArchiveDir;
     this.leaseRecovery = leaseRecovery;
   }
 
@@ -247,16 +239,6 @@ public class WALProcedureStore extends ProcedureStoreBase {
       }
     };
     syncThread.start();
-
-    // Create archive dir up front. Rename won't work w/o it up on HDFS.
-    if (this.walArchiveDir != null && !this.fs.exists(this.walArchiveDir)) {
-      if (this.fs.mkdirs(this.walArchiveDir)) {
-        if (LOG.isDebugEnabled()) LOG.debug("Created Procedure Store WAL archive dir " +
-            this.walArchiveDir);
-      } else {
-        LOG.warn("Failed create of " + this.walArchiveDir);
-      }
-    }
   }
 
   @Override
@@ -310,9 +292,9 @@ public class WALProcedureStore extends ProcedureStoreBase {
   }
 
   @Override
-  public int setRunningProcedureCount(final int count) {
+  public void setRunningProcedureCount(final int count) {
+    LOG.debug("Set running procedure count=" + count + ", slots=" + slots.length);
     this.runningProcCount = count > 0 ? Math.min(count, slots.length) : slots.length;
-    return this.runningProcCount;
   }
 
   public ProcedureStoreTracker getStoreTracker() {
@@ -361,7 +343,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Someone else created new logs. Expected maxLogId < " + flushLogId);
           }
-          logs.getLast().removeFile(this.walArchiveDir);
+          logs.getLast().removeFile();
           continue;
         }
 
@@ -973,7 +955,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
     // but we should check if someone else has created new files
     if (getMaxLogId(getLogFiles()) > flushLogId) {
       LOG.warn("Someone else created new logs. Expected maxLogId < " + flushLogId);
-      logs.getLast().removeFile(this.walArchiveDir);
+      logs.getLast().removeFile();
       return false;
     }
 
@@ -1065,7 +1047,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
     // We keep track of which procedures are holding the oldest WAL in 'holdingCleanupTracker'.
     // once there is nothing olding the oldest WAL we can remove it.
     while (logs.size() > 1 && holdingCleanupTracker.isEmpty()) {
-      removeLogFile(logs.getFirst(), walArchiveDir);
+      removeLogFile(logs.getFirst());
       buildHoldingCleanupTracker();
     }
 
@@ -1097,8 +1079,8 @@ public class WALProcedureStore extends ProcedureStoreBase {
   private void removeAllLogs(long lastLogId) {
     if (logs.size() <= 1) return;
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Remove all state logs with ID less than " + lastLogId);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Remove all state logs with ID less than " + lastLogId);
     }
 
     boolean removed = false;
@@ -1107,7 +1089,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
       if (lastLogId < log.getLogId()) {
         break;
       }
-      removeLogFile(log, walArchiveDir);
+      removeLogFile(log);
       removed = true;
     }
 
@@ -1116,15 +1098,15 @@ public class WALProcedureStore extends ProcedureStoreBase {
     }
   }
 
-  private boolean removeLogFile(final ProcedureWALFile log, final Path walArchiveDir) {
+  private boolean removeLogFile(final ProcedureWALFile log) {
     try {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Removing log=" + log);
       }
-      log.removeFile(walArchiveDir);
+      log.removeFile();
       logs.remove(log);
       if (LOG.isDebugEnabled()) {
-        LOG.info("Removed log=" + log + ", activeLogs=" + logs);
+        LOG.info("Removed log=" + log + " activeLogs=" + logs);
       }
       assert logs.size() > 0 : "expected at least one log";
     } catch (IOException e) {
@@ -1146,7 +1128,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   }
 
   protected Path getLogFilePath(final long logId) throws IOException {
-    return new Path(walDir, String.format(LOG_PREFIX + "%020d.log", logId));
+    return new Path(walDir, String.format("state-%020d.log", logId));
   }
 
   private static long getLogIdFromName(final String name) {
@@ -1159,7 +1141,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
     @Override
     public boolean accept(Path path) {
       String name = path.getName();
-      return name.startsWith(LOG_PREFIX) && name.endsWith(".log");
+      return name.startsWith("state-") && name.endsWith(".log");
     }
   };
 
@@ -1210,7 +1192,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
         }
 
         maxLogId = Math.max(maxLogId, getLogIdFromName(logPath.getName()));
-        ProcedureWALFile log = initOldLog(logFiles[i], this.walArchiveDir);
+        ProcedureWALFile log = initOldLog(logFiles[i]);
         if (log != null) {
           this.logs.add(log);
         }
@@ -1240,22 +1222,21 @@ public class WALProcedureStore extends ProcedureStoreBase {
   /**
    * Loads given log file and it's tracker.
    */
-  private ProcedureWALFile initOldLog(final FileStatus logFile, final Path walArchiveDir)
-  throws IOException {
+  private ProcedureWALFile initOldLog(final FileStatus logFile) throws IOException {
     final ProcedureWALFile log = new ProcedureWALFile(fs, logFile);
     if (logFile.getLen() == 0) {
       LOG.warn("Remove uninitialized log: " + logFile);
-      log.removeFile(walArchiveDir);
+      log.removeFile();
       return null;
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Opening Pv2 " + logFile);
+      LOG.debug("Opening state-log: " + logFile);
     }
     try {
       log.open();
     } catch (ProcedureWALFormat.InvalidWALDataException e) {
       LOG.warn("Remove uninitialized log: " + logFile, e);
-      log.removeFile(walArchiveDir);
+      log.removeFile();
       return null;
     } catch (IOException e) {
       String msg = "Unable to read state log: " + logFile;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 faf8e7e..cde37bd 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
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-// FIX namings. TODO.
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public final class DelayedUtil {
@@ -149,9 +148,6 @@ public final class DelayedUtil {
     }
   }
 
-  /**
-   * Has a timeout.
-   */
   public static class DelayedContainerWithTimestamp<T> extends DelayedContainer<T> {
     private long timeout;
 
@@ -169,4 +165,4 @@ public final class DelayedUtil {
       this.timeout = timeout;
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
index 78daf5a..408cffd 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
@@ -42,7 +42,7 @@ public class TestProcedureToString {
    */
   static class BasicProcedure extends Procedure<BasicProcedureEnv> {
     @Override
-    protected Procedure<BasicProcedureEnv>[] execute(BasicProcedureEnv env)
+    protected Procedure<?>[] execute(BasicProcedureEnv env)
         throws ProcedureYieldException, InterruptedException {
       return new Procedure [] {this};
     }
@@ -78,6 +78,8 @@ public class TestProcedureToString {
     }
   }
 
+  
+
   /**
    * Test that I can override the toString for its state value.
    * @throws ProcedureYieldException


[11/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..3600fe0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MergeTableRegionsProcedure.java
@@ -0,0 +1,906 @@
+/**
+ * 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.forNumber(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(), true),
+            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/a3c5a744/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 622c19f..52bb4d5 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,14 +21,17 @@ 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;
@@ -94,9 +97,7 @@ public class ModifyColumnFamilyProcedure
         setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
         break;
       case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
-        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
-          addChildProcedure(env.getAssignmentManager().createReopenProcedures(getTableName()));
-        }
+        reOpenAllRegionsIfTableIsOnline(env);
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
@@ -264,8 +265,7 @@ public class ModifyColumnFamilyProcedure
     env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    //reOpenAllRegionsIfTableIsOnline(env);
-    // TODO: NUKE ROLLBACK!!!!
+    reOpenAllRegionsIfTableIsOnline(env);
   }
 
   /**
@@ -281,6 +281,26 @@ 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/a3c5a744/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 20a6a03..6a70f62 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,10 +120,7 @@ public class ModifyTableProcedure
         setNextState(ModifyTableState.MODIFY_TABLE_REOPEN_ALL_REGIONS);
         break;
       case MODIFY_TABLE_REOPEN_ALL_REGIONS:
-        if (env.getAssignmentManager().isTableEnabled(getTableName())) {
-          addChildProcedure(env.getAssignmentManager()
-            .createReopenProcedures(getRegionInfoList(env)));
-        }
+        reOpenAllRegionsIfTableIsOnline(env);
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -302,8 +299,7 @@ public class ModifyTableProcedure
     deleteFromFs(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
 
     // Make sure regions are opened after table descriptor is updated.
-    //reOpenAllRegionsIfTableIsOnline(env);
-    // TODO: NUKE ROLLBACK!!!!
+    reOpenAllRegionsIfTableIsOnline(env);
   }
 
   /**
@@ -378,6 +374,25 @@ 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
@@ -415,8 +430,7 @@ public class ModifyTableProcedure
 
   private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
     if (regionInfoList == null) {
-      regionInfoList = env.getAssignmentManager().getRegionStates()
-          .getRegionsOfTable(getTableName());
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
     }
     return regionInfoList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 5199bf8..3777c79 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,26 +21,30 @@ 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.assignment.RegionStates;
+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.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.
@@ -60,93 +64,19 @@ 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 {
-    if (proc.isInitializing()) {
-      procExec.submitProcedure(proc);
-    }
-    return waitForProcedureToCompleteIOE(procExec, proc.getProcId(), Long.MAX_VALUE);
+    long procId = procExec.submitProcedure(proc);
+    return waitForProcedureToComplete(procExec, procId);
   }
 
-  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);
+  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[] waitForProcedureToComplete(
-      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId, final long timeout)
-      throws IOException {
-    waitFor(procExec.getEnvironment(), "pid=" + 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()) {
@@ -156,7 +86,7 @@ public final class ProcedureSyncWait {
       return result.getResult();
     } else {
       if (procExec.isRunning()) {
-        throw new IOException("pid= " + procId + "not found");
+        throw new IOException("Procedure " + procId + "not found");
       } else {
         throw new IOException("The Master is Aborting");
       }
@@ -174,7 +104,6 @@ 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)) {
@@ -186,12 +115,7 @@ public final class ProcedureSyncWait {
         LOG.warn("Interrupted while sleeping, waiting on " + purpose);
         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("waitFor " + purpose);
-      } else {
-        if (!logged) LOG.debug("waitFor " + purpose);
-      }
-      logged = true;
+      LOG.debug("Waiting on " + purpose);
     } while (EnvironmentEdgeManager.currentTime() < done && env.isRunning());
 
     throw new TimeoutIOException("Timed out while waiting on " + purpose);
@@ -209,14 +133,44 @@ 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 RegionStates states = env.getAssignmentManager().getRegionStates();
+    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    final RegionStates states = am.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/a3c5a744/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
deleted file mode 100644
index 887e272..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ /dev/null
@@ -1,541 +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 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(
-        "Using procedure batch rpc execution for serverName=%s version=%s",
-        serverName, rsVersion));
-      submitTask(new ExecuteProceduresRemoteCall(serverName, operations));
-    } else {
-      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/a3c5a744/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 cfd9df9..21709f8 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,6 +43,7 @@ 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;
@@ -415,7 +416,17 @@ public class RestoreSnapshotProcedure
     try {
       Connection conn = env.getMasterServices().getConnection();
 
-      // 1. Prepare to restore
+      // 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);
+
       getMonitorStatus().setStatus("Preparing to restore each region");
 
       // 2. Applies changes to hbase:meta
@@ -485,6 +496,20 @@ 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
deleted file mode 100644
index ca351f6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashException.java
+++ /dev/null
@@ -1,46 +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 org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Passed as Exception by {@link ServerCrashProcedure}
- * notifying on-going RIT that server has failed.
- */
-@InterfaceAudience.Private
-@SuppressWarnings("serial")
-public class ServerCrashException extends HBaseIOException {
-  private final long procId;
-  private final ServerName serverName;
-
-  /**
-   * @param serverName The server that crashed.
-   */
-  public ServerCrashException(long procId, ServerName serverName) {
-    this.procId = procId;
-    this.serverName = serverName;
-  }
-
-  @Override
-  public String getMessage() {
-    return "ServerCrashProcedure pid=" + this.procId + ", server=" + this.serverName;
-  }
-}
\ No newline at end of file


[27/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)"
Revert a mistaken commit!!!

This reverts commit dc1065a85da3098f4defbe008b08057e34d94d4f.


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

Branch: refs/heads/master
Commit: a3c5a74487fa47b6cb8b2132e7fdc5f0f4f7f314
Parents: dc1065a
Author: Michael Stack <st...@apache.org>
Authored: Wed May 24 23:31:36 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed May 24 23:31:36 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ClusterStatus.java  |     8 +-
 .../org/apache/hadoop/hbase/HRegionInfo.java    |    14 -
 .../apache/hadoop/hbase/MetaTableAccessor.java  |    15 +-
 .../hbase/client/ConnectionImplementation.java  |    12 -
 .../client/ShortCircuitMasterConnection.java    |    12 -
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java |     4 +-
 .../apache/hadoop/hbase/ipc/RpcConnection.java  |     6 +-
 .../apache/hadoop/hbase/master/RegionState.java |    22 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   122 +-
 .../hbase/shaded/protobuf/RequestConverter.java |    16 +-
 .../shaded/protobuf/ResponseConverter.java      |    13 +
 .../hbase/zookeeper/MetaTableLocator.java       |     7 +-
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |     6 +-
 .../master/MetricsAssignmentManagerSource.java  |    20 +-
 .../MetricsAssignmentManagerSourceImpl.java     |    36 +-
 .../procedure2/AbstractProcedureScheduler.java  |    40 +-
 .../hadoop/hbase/procedure2/Procedure.java      |   206 +-
 .../hadoop/hbase/procedure2/ProcedureEvent.java |     2 +-
 .../hbase/procedure2/ProcedureExecutor.java     |   223 +-
 .../procedure2/ProcedureInMemoryChore.java      |     6 +-
 .../hbase/procedure2/ProcedureScheduler.java    |     3 +-
 .../procedure2/RemoteProcedureDispatcher.java   |   375 -
 .../hbase/procedure2/SequentialProcedure.java   |     9 +-
 .../hbase/procedure2/StateMachineProcedure.java |    28 +-
 .../procedure2/store/NoopProcedureStore.java    |     4 +-
 .../hbase/procedure2/store/ProcedureStore.java  |     3 +-
 .../procedure2/store/wal/ProcedureWALFile.java  |    18 +-
 .../store/wal/ProcedureWALFormatReader.java     |    38 +-
 .../procedure2/store/wal/WALProcedureStore.java |    55 +-
 .../hbase/procedure2/util/DelayedUtil.java      |     6 +-
 .../hbase/procedure2/TestProcedureToString.java |     4 +-
 .../protobuf/generated/AccessControlProtos.java |   102 +-
 .../shaded/protobuf/generated/AdminProtos.java  | 17247 +++++++----------
 .../generated/MasterProcedureProtos.java        |  9360 ++-------
 .../shaded/protobuf/generated/MasterProtos.java |  7786 +++-----
 .../shaded/protobuf/generated/QuotaProtos.java  |   134 +-
 .../generated/RegionServerStatusProtos.java     |  1679 +-
 .../protobuf/generated/SnapshotProtos.java      |    22 +-
 .../src/main/protobuf/Admin.proto               |    51 +-
 .../src/main/protobuf/Master.proto              |    37 -
 .../src/main/protobuf/MasterProcedure.proto     |   117 +-
 .../src/main/protobuf/RegionServerStatus.proto  |    26 +
 .../hbase/rsgroup/RSGroupAdminServer.java       |    13 +-
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java |     9 +-
 .../balancer/TestRSGroupBasedLoadBalancer.java  |     2 +-
 .../hadoop/hbase/rsgroup/TestRSGroups.java      |    16 +-
 .../hbase/rsgroup/TestRSGroupsOfflineMode.java  |     3 +-
 .../master/AssignmentManagerStatusTmpl.jamon    |    51 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |     2 +-
 .../hadoop/hbase/RegionStateListener.java       |     7 +-
 .../org/apache/hadoop/hbase/SplitLogTask.java   |     4 -
 .../hadoop/hbase/backup/HFileArchiver.java      |    15 +-
 .../hadoop/hbase/client/VersionInfoUtil.java    |    81 +-
 .../hbase/coprocessor/RegionObserver.java       |    22 +-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java |     9 +-
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    |     5 +-
 .../hadoop/hbase/ipc/SimpleRpcServer.java       |    16 +-
 .../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     |   101 +-
 .../apache/hadoop/hbase/master/DeadServer.java  |     6 +-
 .../hbase/master/GeneralBulkAssigner.java       |   213 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   209 +-
 .../hadoop/hbase/master/LoadBalancer.java       |     2 +-
 .../hbase/master/MasterCoprocessorHost.java     |    22 -
 .../hadoop/hbase/master/MasterDumpServlet.java  |     8 +-
 .../hbase/master/MasterMetaBootstrap.java       |    61 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   109 +-
 .../hadoop/hbase/master/MasterServices.java     |    30 -
 .../hadoop/hbase/master/MasterWalManager.java   |    17 +-
 .../hbase/master/MetricsAssignmentManager.java  |    39 +-
 .../hbase/master/NoSuchProcedureException.java  |    33 -
 .../apache/hadoop/hbase/master/RegionPlan.java  |     4 +-
 .../hadoop/hbase/master/RegionStateStore.java   |   268 +
 .../hadoop/hbase/master/RegionStates.java       |  1170 ++
 .../hadoop/hbase/master/ServerManager.java      |    85 +-
 .../hadoop/hbase/master/SplitLogManager.java    |     2 +-
 .../hbase/master/TableNamespaceManager.java     |     5 +-
 .../hadoop/hbase/master/TableStateManager.java  |     3 +-
 .../hadoop/hbase/master/UnAssignCallable.java   |    47 +
 .../master/assignment/AssignProcedure.java      |   338 -
 .../master/assignment/AssignmentManager.java    |  1709 --
 .../FailedRemoteDispatchException.java          |    33 -
 .../assignment/GCMergedRegionsProcedure.java    |   170 -
 .../master/assignment/GCRegionProcedure.java    |   155 -
 .../assignment/MergeTableRegionsProcedure.java  |   776 -
 .../master/assignment/MoveRegionProcedure.java  |   145 -
 .../master/assignment/RegionStateStore.java     |   327 -
 .../hbase/master/assignment/RegionStates.java   |   969 -
 .../assignment/RegionTransitionProcedure.java   |   381 -
 .../assignment/SplitTableRegionProcedure.java   |   733 -
 .../master/assignment/UnassignProcedure.java    |   247 -
 .../hadoop/hbase/master/assignment/Util.java    |    60 -
 .../hbase/master/balancer/BaseLoadBalancer.java |    33 +-
 .../balancer/FavoredStochasticBalancer.java     |    11 +-
 .../master/balancer/RegionLocationFinder.java   |    14 +-
 .../master/balancer/SimpleLoadBalancer.java     |     9 +-
 .../master/balancer/StochasticLoadBalancer.java |     8 +-
 .../hbase/master/locking/LockProcedure.java     |     3 +-
 .../AbstractStateMachineNamespaceProcedure.java |     3 +-
 .../AbstractStateMachineRegionProcedure.java    |   133 -
 .../AbstractStateMachineTableProcedure.java     |    14 +-
 .../procedure/AddColumnFamilyProcedure.java     |    31 +-
 .../procedure/CloneSnapshotProcedure.java       |     4 +-
 .../master/procedure/CreateTableProcedure.java  |    41 +-
 .../procedure/DeleteColumnFamilyProcedure.java  |    31 +-
 .../master/procedure/DeleteTableProcedure.java  |    12 +-
 .../master/procedure/DisableTableProcedure.java |   156 +-
 .../DispatchMergingRegionsProcedure.java        |   584 -
 .../master/procedure/EnableTableProcedure.java  |   172 +-
 .../procedure/MasterDDLOperationHelper.java     |    93 +-
 .../procedure/MasterProcedureConstants.java     |     2 +-
 .../master/procedure/MasterProcedureEnv.java    |    30 +-
 .../procedure/MasterProcedureScheduler.java     |    25 +-
 .../procedure/MergeTableRegionsProcedure.java   |   906 +
 .../procedure/ModifyColumnFamilyProcedure.java  |    30 +-
 .../master/procedure/ModifyTableProcedure.java  |    30 +-
 .../master/procedure/ProcedureSyncWait.java     |   146 +-
 .../master/procedure/RSProcedureDispatcher.java |   541 -
 .../procedure/RestoreSnapshotProcedure.java     |    27 +-
 .../master/procedure/ServerCrashException.java  |    46 -
 .../master/procedure/ServerCrashProcedure.java  |   587 +-
 .../procedure/SplitTableRegionProcedure.java    |   785 +
 .../procedure/TableProcedureInterface.java      |     3 +-
 .../procedure/TruncateTableProcedure.java       |     6 +-
 .../apache/hadoop/hbase/mob/MobFileCache.java   |     4 +-
 .../hbase/namespace/NamespaceAuditor.java       |    10 +-
 .../hbase/namespace/NamespaceStateManager.java  |     5 +-
 .../hadoop/hbase/quotas/MasterQuotaManager.java |     8 +-
 .../hadoop/hbase/regionserver/CompactSplit.java |   723 -
 .../hbase/regionserver/CompactSplitThread.java  |   695 +
 .../regionserver/CompactedHFilesDischarger.java |    77 +-
 .../hadoop/hbase/regionserver/HRegion.java      |    92 +-
 .../hbase/regionserver/HRegionFileSystem.java   |     4 +-
 .../hbase/regionserver/HRegionServer.java       |   134 +-
 .../hbase/regionserver/RSRpcServices.java       |   120 +-
 .../hadoop/hbase/regionserver/Region.java       |     8 -
 .../hbase/regionserver/RegionMergeRequest.java  |   108 -
 .../regionserver/RegionServerServices.java      |    10 +
 .../hbase/regionserver/RegionUnassigner.java    |     5 +-
 .../hadoop/hbase/regionserver/SplitRequest.java |    91 +-
 .../handler/CloseRegionHandler.java             |     2 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |     2 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |    24 +-
 .../apache/hadoop/hbase/wal/WALSplitter.java    |    16 +-
 .../hbase/zookeeper/RegionServerTracker.java    |     4 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |    12 +-
 .../hadoop/hbase/MockRegionServerServices.java  |    10 +
 .../hadoop/hbase/TestRegionRebalancing.java     |    16 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |    20 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |     4 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |    51 +-
 .../client/TestAsyncTableGetMultiThreaded.java  |     2 -
 ...ableGetMultiThreadedWithBasicCompaction.java |     6 +-
 ...ableGetMultiThreadedWithEagerCompaction.java |     6 +-
 .../client/TestBlockEvictionFromClient.java     |     2 -
 .../hadoop/hbase/client/TestEnableTable.java    |    34 +-
 .../hbase/client/TestFromClientSide3.java       |     2 -
 .../org/apache/hadoop/hbase/client/TestHCM.java |   142 +-
 .../hbase/client/TestMetaWithReplicas.java      |    37 +-
 .../client/TestScannersFromClientSide.java      |    30 +-
 .../hbase/client/TestServerBusyException.java   |   234 -
 .../client/TestSnapshotCloneIndependence.java   |     2 +-
 .../hbase/client/TestSplitOrMergeStatus.java    |   119 +-
 .../hbase/client/TestTableFavoredNodes.java     |    53 +-
 .../coprocessor/TestIncrementTimeRange.java     |     5 +-
 .../hbase/ipc/TestSimpleRpcScheduler.java       |     7 +-
 .../mapreduce/TestLoadIncrementalHFiles.java    |     2 +-
 .../hbase/master/MockNoopMasterServices.java    |    22 +-
 .../hadoop/hbase/master/MockRegionServer.java   |    37 +-
 .../hbase/master/TestAssignmentListener.java    |     1 -
 .../master/TestAssignmentManagerOnCluster.java  |  1403 ++
 .../hadoop/hbase/master/TestCatalogJanitor.java |   591 +-
 .../master/TestDistributedLogSplitting.java     |     1 -
 .../apache/hadoop/hbase/master/TestMaster.java  |     1 -
 .../master/TestMasterBalanceThrottling.java     |    11 +-
 .../hadoop/hbase/master/TestMasterFailover.java |    31 +-
 .../hadoop/hbase/master/TestMasterMetrics.java  |     8 +-
 .../TestMasterOperationsForRegionReplicas.java  |    10 +-
 .../hbase/master/TestMasterStatusServlet.java   |    58 +-
 .../hbase/master/TestMasterWalManager.java      |     2 +-
 .../hbase/master/TestMetaShutdownHandler.java   |     1 -
 .../hadoop/hbase/master/TestRegionState.java    |    17 +-
 .../hadoop/hbase/master/TestRegionStates.java   |   144 +
 .../hadoop/hbase/master/TestRestartCluster.java |     8 +-
 .../hadoop/hbase/master/TestWarmupRegion.java   |    14 +-
 .../assignment/AssignmentTestingUtil.java       |   125 -
 .../master/assignment/MockMasterServices.java   |   358 -
 .../assignment/TestAssignmentManager.java       |   750 -
 .../assignment/TestAssignmentOnRSCrash.java     |   185 -
 .../TestMergeTableRegionsProcedure.java         |   260 -
 .../master/assignment/TestRegionStates.java     |   224 -
 .../TestSplitTableRegionProcedure.java          |   428 -
 .../TestFavoredStochasticBalancerPickers.java   |     3 +-
 .../TestFavoredStochasticLoadBalancer.java      |    27 +-
 .../TestSimpleRegionNormalizerOnCluster.java    |     3 +-
 ...ProcedureSchedulerPerformanceEvaluation.java |     2 +-
 .../MasterProcedureTestingUtility.java          |    67 +-
 .../procedure/TestAddColumnFamilyProcedure.java |    34 +-
 .../procedure/TestCloneSnapshotProcedure.java   |     8 +-
 .../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 +-
 .../procedure/TestMasterProcedureScheduler.java |    20 +-
 .../TestModifyColumnFamilyProcedure.java        |     9 +-
 .../procedure/TestModifyNamespaceProcedure.java |     4 +-
 .../procedure/TestModifyTableProcedure.java     |    18 +-
 .../master/procedure/TestProcedureAdmin.java    |    12 +-
 .../procedure/TestRestoreSnapshotProcedure.java |    12 +-
 .../procedure/TestServerCrashProcedure.java     |   115 +-
 .../TestSplitTableRegionProcedure.java          |   420 +
 .../procedure/TestTableDDLProcedureBase.java    |     7 +-
 .../procedure/TestTruncateTableProcedure.java   |    11 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |   184 +-
 .../procedure/SimpleMasterProcedureManager.java |     2 +-
 .../regionserver/TestCompactSplitThread.java    |    21 +-
 .../hbase/regionserver/TestCompaction.java      |    10 +-
 .../TestCorruptedRegionStoreFile.java           |     5 -
 .../regionserver/TestHRegionFileSystem.java     |     6 +-
 .../TestRegionMergeTransactionOnCluster.java    |    58 +-
 .../regionserver/TestRegionServerMetrics.java   |    40 +-
 .../TestSplitTransactionOnCluster.java          |   140 +-
 .../hbase/regionserver/wal/TestLogRolling.java  |     5 -
 .../wal/TestSecureAsyncWALReplay.java           |     5 -
 .../hbase/regionserver/wal/TestWALReplay.java   |     8 +-
 .../security/access/TestAccessController3.java  |    14 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |     4 +-
 .../hadoop/hbase/util/TestHBaseFsckMOB.java     |     2 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |   118 +-
 .../hbase/util/TestHBaseFsckReplicas.java       |     4 +-
 .../hadoop/hbase/util/TestHBaseFsckTwoRS.java   |    23 +-
 .../util/hbck/TestOfflineMetaRebuildBase.java   |     3 +-
 .../util/hbck/TestOfflineMetaRebuildHole.java   |     2 -
 .../hbck/TestOfflineMetaRebuildOverlap.java     |     2 -
 242 files changed, 25902 insertions(+), 37835 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 95d77a2..c51a437 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
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.List;
+import java.util.Set;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -67,7 +67,7 @@ public class ClusterStatus extends VersionedWritable {
   private Collection<ServerName> deadServers;
   private ServerName master;
   private Collection<ServerName> backupMasters;
-  private List<RegionState> intransition;
+  private Set<RegionState> intransition;
   private String clusterId;
   private String[] masterCoprocessors;
   private Boolean balancerOn;
@@ -77,7 +77,7 @@ public class ClusterStatus extends VersionedWritable {
       final Collection<ServerName> deadServers,
       final ServerName master,
       final Collection<ServerName> backupMasters,
-      final List<RegionState> rit,
+      final Set<RegionState> rit,
       final String[] masterCoprocessors,
       final Boolean balancerOn) {
     this.hbaseVersion = hbaseVersion;
@@ -248,7 +248,7 @@ public class ClusterStatus extends VersionedWritable {
   }
 
   @InterfaceAudience.Private
-  public List<RegionState> getRegionsInTransition() {
+  public Set<RegionState> getRegionsInTransition() {
     return this.intransition;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index d470ffa..bc93cc6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -168,19 +167,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     return prettyPrint(this.getEncodedName());
   }
 
-  public static String getShortNameToLog(HRegionInfo...hris) {
-    return getShortNameToLog(Arrays.asList(hris));
-  }
-
-  /**
-   * @return Return a String of short, printable names for <code>hris</code>
-   * (usually encoded name) for us logging.
-   */
-  public static String getShortNameToLog(final List<HRegionInfo> hris) {
-    return hris.stream().map(hri -> hri.getShortNameToLog()).
-        collect(Collectors.toList()).toString();
-  }
-
   /**
    * Use logging.
    * @param encodedRegionName The encoded regionname.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 9eb5111..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
@@ -1663,11 +1663,8 @@ public class MetaTableAccessor {
       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
       Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
 
-      // The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
-      // if crash after merge happened but before we got to here.. means in-memory
-      // locations of offlined merged, now-closed, regions is lost. Should be ok. We
-      // assign the merged region later.
-      if (sn != null) addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
+      // The merged is a new region, openSeqNum = 1 is fine.
+      addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
 
       // Add empty locations for region replicas of the merged region so that number of replicas can
       // be cached whenever the primary region is looked up from meta
@@ -1969,8 +1966,8 @@ public class MetaTableAccessor {
    * @param regionsInfo list of regions to be deleted from META
    * @throws IOException
    */
-  public static void deleteRegions(Connection connection, List<HRegionInfo> regionsInfo, long ts)
-  throws IOException {
+  public static void deleteRegions(Connection connection,
+                                   List<HRegionInfo> regionsInfo, long ts) throws IOException {
     List<Delete> deletes = new ArrayList<>(regionsInfo.size());
     for (HRegionInfo hri: regionsInfo) {
       Delete e = new Delete(hri.getRegionName());
@@ -2005,10 +2002,10 @@ public class MetaTableAccessor {
     }
     mutateMetaTable(connection, mutation);
     if (regionsToRemove != null && regionsToRemove.size() > 0) {
-      LOG.debug("Deleted " + HRegionInfo.getShortNameToLog(regionsToRemove));
+      LOG.debug("Deleted " + regionsToRemove);
     }
     if (regionsToAdd != null && regionsToAdd.size() > 0) {
-      LOG.debug("Added " + HRegionInfo.getShortNameToLog(regionsToAdd));
+      LOG.debug("Added " + regionsToAdd);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 4ed28ec..e5f5694 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
@@ -1339,12 +1339,6 @@ 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 {
@@ -1364,12 +1358,6 @@ 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/a3c5a744/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 6d75446..bea578c 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
@@ -499,16 +499,4 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
       GetQuotaStatesRequest request) throws ServiceException {
     return stub.getQuotaStates(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);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
index 08533b4..e69b42d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -226,8 +226,8 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
       switch (idleEvt.state()) {
         case WRITER_IDLE:
           if (id2Call.isEmpty()) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("shutdown connection to " + conn.remoteId().address
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("shutdown connection to " + conn.remoteId().address
                   + " because idle for a long time");
             }
             // It may happen that there are still some pending calls in the event loop queue and

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
index 98d2256..b5a7959 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
@@ -129,11 +129,7 @@ abstract class RpcConnection {
       authMethod = AuthMethod.KERBEROS;
     }
 
-    // Log if debug AND non-default auth, else if trace enabled.
-    // No point logging obvious.
-    if ((LOG.isDebugEnabled() && !authMethod.equals(AuthMethod.SIMPLE)) ||
-        LOG.isTraceEnabled()) {
-      // Only log if not default auth.
+    if (LOG.isDebugEnabled()) {
       LOG.debug("Use " + authMethod + " authentication for service " + remoteId.serviceName
           + ", sasl=" + useSasl);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 7116763..0e12ef6 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,8 +36,10 @@ 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
@@ -62,12 +64,18 @@ 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;
@@ -116,6 +124,8 @@ public class RegionState {
         state = OFFLINE;
         break;
       case PENDING_OPEN:
+        state = PENDING_OPEN;
+        break;
       case OPENING:
         state = OPENING;
         break;
@@ -123,6 +133,8 @@ public class RegionState {
         state = OPEN;
         break;
       case PENDING_CLOSE:
+        state = PENDING_CLOSE;
+        break;
       case CLOSING:
         state = CLOSING;
         break;
@@ -219,16 +231,22 @@ public class RegionState {
     this.ritDuration += (this.stamp - previousStamp);
   }
 
+  /**
+   * PENDING_CLOSE (to be removed) is the same as CLOSING
+   */
   public boolean isClosing() {
-    return state == State.CLOSING;
+    return state == State.PENDING_CLOSE || 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.OPENING;
+    return state == State.PENDING_OPEN || state == State.OPENING;
   }
 
   public boolean isOpened() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 eca050f..108646a 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;
 
@@ -89,14 +89,12 @@ import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.procedure2.LockInfo;
 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.SpaceViolationPolicy;
 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;
@@ -110,6 +108,8 @@ 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;
@@ -177,7 +177,6 @@ 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.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
@@ -1843,6 +1842,33 @@ 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
    *
@@ -1994,46 +2020,6 @@ 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
 
   /*
@@ -3117,8 +3103,8 @@ public final class ProtobufUtil {
       backupMasters.add(ProtobufUtil.toServerName(sn));
     }
 
-    List<RegionState> rit =
-      new ArrayList<>(proto.getRegionsInTransitionList().size());
+    Set<RegionState> rit = null;
+    rit = new HashSet<>(proto.getRegionsInTransitionList().size());
     for (RegionInTransition region : proto.getRegionsInTransitionList()) {
       RegionState value = RegionState.convert(region.getRegionState());
       rit.add(value);
@@ -3277,6 +3263,26 @@ 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
@@ -3325,28 +3331,6 @@ 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 134c319..4d34334 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
@@ -123,6 +123,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaSta
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaRegionSizesRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetSpaceQuotaSnapshotsRequest;
 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;
@@ -1119,6 +1120,19 @@ 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
    *
@@ -1501,7 +1515,7 @@ public final class RequestConverter {
   /**
    * Create a RegionOpenInfo based on given region info and version of offline node
    */
-  public static RegionOpenInfo buildRegionOpenInfo(
+  private 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/a3c5a744/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 c489628..ecadbbc 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,6 +34,7 @@ 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;
@@ -253,6 +254,18 @@ 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/a3c5a744/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index c11d896..afab54a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -439,10 +439,6 @@ public class MetaTableLocator {
    */
   public static void setMetaLocation(ZooKeeperWatcher zookeeper,
       ServerName serverName, int replicaId, RegionState.State state) throws KeeperException {
-    if (serverName == null) {
-      LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
-      return;
-    }
     LOG.info("Setting hbase:meta region location in ZooKeeper as " + serverName);
     // Make the MetaRegionServer pb and then get its bytes and save this as
     // the znode content.
@@ -452,8 +448,7 @@ public class MetaTableLocator {
       .setState(state.convert()).build();
     byte[] data = ProtobufUtil.prependPBMagic(pbrsr.toByteArray());
     try {
-      ZKUtil.setData(zookeeper,
-          zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
+      ZKUtil.setData(zookeeper, zookeeper.znodePaths.getZNodeForReplica(replicaId), data);
     } catch(KeeperException.NoNodeException nne) {
       if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
         LOG.debug("META region location doesn't exist, create it");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index 36dabdd..6104c22 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -80,11 +80,12 @@ public class ProcedureInfo implements Cloneable {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
+    sb.append("Procedure=");
     sb.append(procName);
-    sb.append(" pid=");
+    sb.append(" (id=");
     sb.append(procId);
     if (hasParentId()) {
-      sb.append(", ppid=");
+      sb.append(", parent=");
       sb.append(parentId);
     }
     if (hasOwner()) {
@@ -106,6 +107,7 @@ public class ProcedureInfo implements Cloneable {
       sb.append(this.exception.getMessage());
       sb.append("\"");
     }
+    sb.append(")");
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 2ebf8c9..fa7bbec 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
@@ -47,7 +47,6 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   String RIT_OLDEST_AGE_NAME = "ritOldestAge";
   String RIT_DURATION_NAME = "ritDuration";
   String ASSIGN_TIME_NAME = "assign";
-  String UNASSIGN_TIME_NAME = "unassign";
   String BULK_ASSIGN_TIME_NAME = "bulkAssign";
 
   String RIT_COUNT_DESC = "Current number of Regions In Transition (Gauge).";
@@ -57,7 +56,9 @@ public interface MetricsAssignmentManagerSource extends BaseSource {
   String RIT_DURATION_DESC =
       "Total durations in milliseconds for all Regions in Transition (Histogram).";
 
-  String OPERATION_COUNT_NAME = "operationCount";
+  void updateAssignmentTime(long time);
+
+  void updateBulkAssignTime(long time);
 
   /**
    * Set the number of regions in transition.
@@ -81,19 +82,4 @@ 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/a3c5a744/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 14b7e71..faae044 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,7 +21,6 @@ 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
@@ -33,10 +32,8 @@ public class MetricsAssignmentManagerSourceImpl
   private MutableGaugeLong ritCountOverThresholdGauge;
   private MutableGaugeLong ritOldestAgeGauge;
   private MetricHistogram ritDurationHisto;
-
-  private MutableFastCounter operationCounter;
   private MetricHistogram assignTimeHisto;
-  private MetricHistogram unassignTimeHisto;
+  private MetricHistogram bulkAssignTimeHisto;
 
   public MetricsAssignmentManagerSourceImpl() {
     this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
@@ -54,39 +51,30 @@ public class MetricsAssignmentManagerSourceImpl
         RIT_COUNT_OVER_THRESHOLD_DESC,0l);
     ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, RIT_OLDEST_AGE_DESC, 0l);
     assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
-    unassignTimeHisto = metricsRegistry.newTimeHistogram(UNASSIGN_TIME_NAME);
+    bulkAssignTimeHisto = metricsRegistry.newTimeHistogram(BULK_ASSIGN_TIME_NAME);
     ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME, RIT_DURATION_DESC);
-    operationCounter = metricsRegistry.getCounter(OPERATION_COUNT_NAME, 0l);
-  }
-
-  @Override
-  public void setRIT(final int ritCount) {
-    ritGauge.set(ritCount);
   }
 
   @Override
-  public void setRITCountOverThreshold(final int ritCount) {
-    ritCountOverThresholdGauge.set(ritCount);
+  public void updateAssignmentTime(long time) {
+    assignTimeHisto.add(time);
   }
 
   @Override
-  public void setRITOldestAge(final long ritCount) {
-    ritOldestAgeGauge.set(ritCount);
+  public void updateBulkAssignTime(long time) {
+    bulkAssignTimeHisto.add(time);
   }
 
-  @Override
-  public void incrementOperationCounter() {
-    operationCounter.incr();
+  public void setRIT(int ritCount) {
+    ritGauge.set(ritCount);
   }
 
-  @Override
-  public void updateAssignTime(final long time) {
-    assignTimeHisto.add(time);
+  public void setRITCountOverThreshold(int ritCount) {
+    ritCountOverThresholdGauge.set(ritCount);
   }
 
-  @Override
-  public void updateUnassignTime(final long time) {
-    unassignTimeHisto.add(time);
+  public void setRITOldestAge(long ritCount) {
+    ritOldestAgeGauge.set(ritCount);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 64c3e53..fbb066c 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
@@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 @InterfaceAudience.Private
 public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   private static final Log LOG = LogFactory.getLog(AbstractProcedureScheduler.class);
-  private final ReentrantLock schedulerLock = new ReentrantLock();
-  private final Condition schedWaitCond = schedulerLock.newCondition();
+  private final ReentrantLock schedLock = new ReentrantLock();
+  private final Condition schedWaitCond = schedLock.newCondition();
   private boolean running = false;
 
   // TODO: metrics
@@ -88,14 +88,14 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   }
 
   protected void push(final Procedure procedure, final boolean addFront, final boolean notify) {
-    schedulerLock.lock();
+    schedLock.lock();
     try {
       enqueue(procedure, addFront);
       if (notify) {
         schedWaitCond.signal();
       }
     } finally {
-      schedulerLock.unlock();
+      schedLock.unlock();
     }
   }
 
@@ -219,11 +219,11 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
 
   @Override
   public void suspendEvent(final ProcedureEvent event) {
-    final boolean traceEnabled = LOG.isTraceEnabled();
+    final boolean isTraceEnabled = LOG.isTraceEnabled();
     synchronized (event) {
       event.setReady(false);
-      if (traceEnabled) {
-        LOG.trace("Suspend " + event);
+      if (isTraceEnabled) {
+        LOG.trace("Suspend event " + event);
       }
     }
   }
@@ -235,29 +235,18 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
 
   @Override
   public void wakeEvents(final int count, final ProcedureEvent... events) {
-    final boolean traceEnabled = LOG.isTraceEnabled();
+    final boolean isTraceEnabled = LOG.isTraceEnabled();
     schedLock();
     try {
       int waitingCount = 0;
       for (int i = 0; i < count; ++i) {
         final ProcedureEvent event = events[i];
         synchronized (event) {
-          if (!event.isReady()) {
-            // Only set ready if we were not ready; i.e. suspended. Otherwise, we double-wake
-            // on this event and down in wakeWaitingProcedures, we double decrement this
-            // finish which messes up child procedure accounting.
-            event.setReady(true);
-            if (traceEnabled) {
-              LOG.trace("Unsuspend " + event);
-            }
-            waitingCount += wakeWaitingProcedures(event.getSuspendedProcedures());
-          } else {
-            ProcedureDeque q = event.getSuspendedProcedures();
-            if (q != null && !q.isEmpty()) {
-              LOG.warn("Q is not empty! size=" + q.size() + "; PROCESSING...");
-              waitingCount += wakeWaitingProcedures(event.getSuspendedProcedures());
-            }
+          event.setReady(true);
+          if (isTraceEnabled) {
+            LOG.trace("Wake event " + event);
           }
+          waitingCount += wakeWaitingProcedures(event.getSuspendedProcedures());
         }
       }
       wakePollIfNeeded(waitingCount);
@@ -286,7 +275,6 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   }
 
   protected void wakeProcedure(final Procedure procedure) {
-    if (LOG.isTraceEnabled()) LOG.trace("Wake " + procedure);
     push(procedure, /* addFront= */ true, /* notify= */false);
   }
 
@@ -294,11 +282,11 @@ public abstract class AbstractProcedureScheduler implements ProcedureScheduler {
   //  Internal helpers
   // ==========================================================================
   protected void schedLock() {
-    schedulerLock.lock();
+    schedLock.lock();
   }
 
   protected void schedUnlock() {
-    schedulerLock.unlock();
+    schedLock.unlock();
   }
 
   protected void wakePollIfNeeded(final int waitingCount) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 09f8170..591c0d0 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
@@ -25,8 +25,6 @@ import java.util.Arrays;
 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.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
@@ -39,66 +37,37 @@ import org.apache.hadoop.hbase.util.NonceKey;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * Base Procedure class responsible for Procedure Metadata;
- * e.g. state, submittedTime, lastUpdate, stack-indexes, etc.
+ * Base Procedure class responsible to handle the Procedure Metadata
+ * e.g. state, submittedTime, lastUpdate, stack-indexes, ...
  *
- * <p>Procedures are run by a {@link ProcedureExecutor} instance. They are submitted and then
- * the ProcedureExecutor keeps calling {@link #execute(Object)} until the Procedure is done.
- * Execute may be called multiple times in the case of failure or a restart, so code must be
- * idempotent. The return from an execute call is either: null to indicate we are done;
- * ourself if there is more to do; or, a set of sub-procedures that need to
- * be run to completion before the framework resumes our execution.
+ * execute() is called each time the procedure is executed.
+ * it may be called multiple times in case of failure and restart, so the
+ * code must be idempotent.
+ * the return is a set of sub-procedures or null in case the procedure doesn't
+ * have sub-procedures. Once the sub-procedures are successfully completed
+ * the execute() method is called again, you should think at it as a stack:
+ *  -&gt; step 1
+ *  ---&gt; step 2
+ *  -&gt; step 1
  *
- * <p>The ProcedureExecutor keeps its
- * notion of Procedure State in the Procedure itself; e.g. it stamps the Procedure as INITIALIZING,
- * RUNNABLE, SUCCESS, etc. Here are some of the States defined in the ProcedureState enum from
- * protos:
- *<ul>
- * <li>{@link #isFailed()} A procedure has executed at least once and has failed. The procedure
- * may or may not have rolled back yet. Any procedure in FAILED state will be eventually moved
- * to ROLLEDBACK state.</li>
- *
- * <li>{@link #isSuccess()} A procedure is completed successfully without exception.</li>
- *
- * <li>{@link #isFinished()} As a procedure in FAILED state will be tried forever for rollback, only
- * condition when scheduler/ executor will drop procedure from further processing is when procedure
- * state is ROLLEDBACK or isSuccess() returns true. This is a terminal state of the procedure.</li>
- *
- * <li>{@link #isWaiting()} - Procedure is in one of the two waiting states
- * ({@link ProcedureState#WAITING}, {@link ProcedureState#WAITING_TIMEOUT}).</li>
- *</ul>
- * NOTE: This states are of the ProcedureExecutor. Procedure implementations in turn can keep
- * their own state. This can lead to confusion. Try to keep the two distinct.
- *
- * <p>rollback() is called when the procedure or one of the sub-procedures
- * has failed. The rollback step is supposed to cleanup the resources created
- * during the execute() step. In case of failure and restart, rollback() may be
- * called multiple times, so again the code must be idempotent.
- *
- * <p>Procedure can be made respect a locking regime. It has acqure/release methods as
- * well as an {@link #hasLock(Object)}. The lock implementation is up to the implementor.
- * If an entity needs to be locked for the life of a procedure -- not just the calls to
- * execute -- then implementations should say so with the {@link #holdLock(Object)}
- * method.
- *
- * <p>There are hooks for collecting metrics on submit of the procedure and on finish.
- * See {@link #updateMetricsOnSubmit(Object)} and
- * {@link #updateMetricsOnFinish(Object, long, boolean)}.
+ * rollback() is called when the procedure or one of the sub-procedures is failed.
+ * the rollback step is supposed to cleanup the resources created during the
+ * execute() step. in case of failure and restart rollback() may be called
+ * multiple times, so the code must be idempotent.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TEnvironment>> {
-  private static final Log LOG = LogFactory.getLog(Procedure.class);
+public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
   public static final long NO_PROC_ID = -1;
   protected static final int NO_TIMEOUT = -1;
 
   public enum LockState {
-    LOCK_ACQUIRED,       // Lock acquired and ready to execute
-    LOCK_YIELD_WAIT,     // Lock not acquired, framework needs to yield
-    LOCK_EVENT_WAIT,     // Lock not acquired, an event will yield the procedure
+    LOCK_ACQUIRED,       // lock acquired and ready to execute
+    LOCK_YIELD_WAIT,     // lock not acquired, framework needs to yield
+    LOCK_EVENT_WAIT,     // lock not acquired, an event will yield the procedure
   }
 
-  // Unchanged after initialization
+  // unchanged after initialization
   private NonceKey nonceKey = null;
   private String owner = null;
   private long parentProcId = NO_PROC_ID;
@@ -106,7 +75,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   private long procId = NO_PROC_ID;
   private long submittedTime;
 
-  // Runtime state, updated every operation
+  // runtime state, updated every operation
   private ProcedureState state = ProcedureState.INITIALIZING;
   private RemoteProcedureException exception = null;
   private int[] stackIndexes = null;
@@ -119,22 +88,19 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * The main code of the procedure. It must be idempotent since execute()
-   * may be called multiple times in case of machine failure in the middle
+   * may be called multiple time in case of machine failure in the middle
    * of the execution.
    * @param env the environment passed to the ProcedureExecutor
-   * @return a set of sub-procedures to run or ourselves if there is more work to do or null if the
-   * procedure is done.
-   * @throws ProcedureYieldException the procedure will be added back to the queue and retried later.
-   * @throws InterruptedException the procedure will be added back to the queue and retried later.
-   * @throws ProcedureSuspendedException Signal to the executor that Procedure has suspended itself and
-   * has set itself up waiting for an external event to wake it back up again.
-   */
-  protected abstract Procedure<TEnvironment>[] execute(TEnvironment env)
+   * @return a set of sub-procedures or null if there is nothing else to execute.
+   * @throws ProcedureYieldException the procedure will be added back to the queue and retried later
+   * @throws InterruptedException the procedure will be added back to the queue and retried later
+   */
+  protected abstract Procedure[] execute(TEnvironment env)
     throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException;
 
   /**
-   * The code to undo what was done by the execute() code.
-   * It is called when the procedure or one of the sub-procedures failed or an
+   * The code to undo what done by the execute() code.
+   * It is called when the procedure or one of the sub-procedure failed or an
    * abort was requested. It should cleanup all the resources created by
    * the execute() call. The implementation must be idempotent since rollback()
    * may be called multiple time in case of machine failure in the middle
@@ -148,21 +114,21 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * The abort() call is asynchronous and each procedure must decide how to deal
-   * with it, if they want to be abortable. The simplest implementation
+   * with that, if they want to be abortable. The simplest implementation
    * is to have an AtomicBoolean set in the abort() method and then the execute()
    * will check if the abort flag is set or not.
    * abort() may be called multiple times from the client, so the implementation
    * must be idempotent.
    *
-   * <p>NOTE: abort() is not like Thread.interrupt(). It is just a notification
-   * that allows the procedure implementor abort.
+   * NOTE: abort() is not like Thread.interrupt() it is just a notification
+   * that allows the procedure implementor where to abort to avoid leak and
+   * have a better control on what was executed and what not.
    */
   protected abstract boolean abort(TEnvironment env);
 
   /**
    * The user-level code of the procedure may have some state to
-   * persist (e.g. input arguments or current position in the processing state) to
-   * be able to resume on failure.
+   * persist (e.g. input arguments) to be able to resume on failure.
    * @param stream the stream that will contain the user serialized data
    */
   protected abstract void serializeStateData(final OutputStream stream)
@@ -177,17 +143,11 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     throws IOException;
 
   /**
-   * The user should override this method if they need a lock on an Entity.
-   * A lock can be anything, and it is up to the implementor. The Procedure
-   * Framework will call this method just before it invokes {@link #execute(Object)}.
-   * It calls {@link #releaseLock(Object)} after the call to execute.
-   * 
-   * <p>If you need to hold the lock for the life of the Procdure -- i.e. you do not
-   * want any other Procedure interfering while this Procedure is running, see
-   * {@link #holdLock(Object)}.
+   * The user should override this method, and try to take a lock if necessary.
+   * A lock can be anything, and it is up to the implementor.
    *
    * <p>Example: in our Master we can execute request in parallel for different tables.
-   * We can create t1 and create t2 and these creates can be executed at the same time.
+   * We can create t1 and create t2 and this can be executed at the same time.
    * Anything else on t1/t2 is queued waiting that specific table create to happen.
    *
    * <p>There are 3 LockState:
@@ -213,9 +173,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * Used to keep the procedure lock even when the procedure is yielding or suspended.
-   * Must implement {@link #hasLock(Object)} if you want to hold the lock for life
-   * of the Procedure.
-   * @see #hasLock(Object)
    * @return true if the procedure should hold on the lock until completionCleanup()
    */
   protected boolean holdLock(final TEnvironment env) {
@@ -223,11 +180,8 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   /**
-   * This is used in conjunction with {@link #holdLock(Object)}. If {@link #holdLock(Object)}
-   * returns true, the procedure executor will call acquireLock() once and thereafter
-   * not call {@link #releaseLock(Object)} until the Procedure is done (Normally, it calls
-   * release/acquire around each invocation of {@link #execute(Object)}.
-   * @see #holdLock(Object)
+   * This is used in conjuction with holdLock(). If holdLock() is true
+   * the procedure executor will not call acquireLock() if hasLock() is true.
    * @return true if the procedure has the lock, false otherwise.
    */
   protected boolean hasLock(final TEnvironment env) {
@@ -255,15 +209,14 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   /**
    * Called when the procedure is marked as completed (success or rollback).
    * The procedure implementor may use this method to cleanup in-memory states.
-   * This operation will not be retried on failure. If a procedure took a lock,
-   * it will have been released when this method runs.
+   * This operation will not be retried on failure.
    */
   protected void completionCleanup(final TEnvironment env) {
     // no-op
   }
 
   /**
-   * By default, the procedure framework/executor will try to run procedures start to finish.
+   * By default, the executor will try to run procedures start to finish.
    * Return true to make the executor yield between each execution step to
    * give other procedures a chance to run.
    * @param env the environment passed to the ProcedureExecutor
@@ -322,30 +275,27 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   protected StringBuilder toStringSimpleSB() {
     final StringBuilder sb = new StringBuilder();
 
-    sb.append("pid=");
+    sb.append("procId=");
     sb.append(getProcId());
 
     if (hasParent()) {
-      sb.append(", ppid=");
+      sb.append(", parentProcId=");
       sb.append(getParentProcId());
     }
 
-    /**
-     * Enable later when this is being used.
-     * Currently owner not used.
     if (hasOwner()) {
       sb.append(", owner=");
       sb.append(getOwner());
-    }*/
+    }
 
-    sb.append(", state="); // pState for Procedure State as opposed to any other kind.
+    sb.append(", state=");
     toStringState(sb);
 
     if (hasException()) {
       sb.append(", exception=" + getException());
     }
 
-    sb.append("; ");
+    sb.append(", ");
     toStringClassDetails(sb);
 
     return sb;
@@ -361,7 +311,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
     sb.append(" submittedTime=");
     sb.append(getSubmittedTime());
 
-    sb.append(", lastUpdate=");
+    sb.append(" lastUpdate=");
     sb.append(getLastUpdate());
 
     final int[] stackIndices = getStackIndexes();
@@ -381,8 +331,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   /**
-   * Called from {@link #toString()} when interpolating {@link Procedure} State.
-   * Allows decorating generic Procedure State with Procedure particulars.
+   * Called from {@link #toString()} when interpolating {@link Procedure} state
    * @param builder Append current {@link ProcedureState}
    */
   protected void toStringState(StringBuilder builder) {
@@ -578,6 +527,25 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   // ==============================================================================================
 
   /**
+   * Procedure has states which are defined in proto file. At some places in the code, we
+   * need to determine more about those states. Following Methods help determine:
+   *
+   * {@link #isFailed()} - A procedure has executed at least once and has failed. The procedure
+   *                       may or may not have rolled back yet. Any procedure in FAILED state
+   *                       will be eventually moved to ROLLEDBACK state.
+   *
+   * {@link #isSuccess()} - A procedure is completed successfully without any exception.
+   *
+   * {@link #isFinished()} - As a procedure in FAILED state will be tried forever for rollback, only
+   *                         condition when scheduler/ executor will drop procedure from further
+   *                         processing is when procedure state is ROLLEDBACK or isSuccess()
+   *                         returns true. This is a terminal state of the procedure.
+   *
+   * {@link #isWaiting()} - Procedure is in one of the two waiting states ({@link
+   *                        ProcedureState#WAITING}, {@link ProcedureState#WAITING_TIMEOUT}).
+   */
+
+  /**
    * @return true if the procedure is in a RUNNABLE state.
    */
   protected synchronized boolean isRunnable() {
@@ -680,10 +648,6 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   @InterfaceAudience.Private
   protected synchronized void setChildrenLatch(final int numChildren) {
     this.childrenLatch = numChildren;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("CHILD LATCH INCREMENT SET " +
-          this.childrenLatch, new Throwable(this.toString()));
-    }
   }
 
   /**
@@ -693,34 +657,15 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   protected synchronized void incChildrenLatch() {
     // TODO: can this be inferred from the stack? I think so...
     this.childrenLatch++;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("CHILD LATCH INCREMENT " + this.childrenLatch, new Throwable(this.toString()));
-    }
   }
 
   /**
    * Called by the ProcedureExecutor to notify that one of the sub-procedures has completed.
    */
   @InterfaceAudience.Private
-  private synchronized boolean childrenCountDown() {
+  protected synchronized boolean childrenCountDown() {
     assert childrenLatch > 0: this;
-    boolean b = --childrenLatch == 0;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("CHILD LATCH DECREMENT " + childrenLatch, new Throwable(this.toString()));
-    }
-    return b;
-  }
-
-  /**
-   * Try to set this procedure into RUNNABLE state.
-   * Succeeds if all subprocedures/children are done.
-   * @return True if we were able to move procedure to RUNNABLE state.
-   */
-  synchronized boolean tryRunnable() {
-    // Don't use isWaiting in the below; it returns true for WAITING and WAITING_TIMEOUT
-    boolean b = getState() == ProcedureState.WAITING && childrenCountDown();
-    if (b) setState(ProcedureState.RUNNABLE);
-    return b;
+    return --childrenLatch == 0;
   }
 
   @InterfaceAudience.Private
@@ -787,11 +732,9 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
 
   /**
    * Internal method called by the ProcedureExecutor that starts the user-level code execute().
-   * @throws ProcedureSuspendedException This is used when procedure wants to halt processing and
-   * skip out without changing states or releasing any locks held.
    */
   @InterfaceAudience.Private
-  protected Procedure<TEnvironment>[] doExecute(final TEnvironment env)
+  protected Procedure[] doExecute(final TEnvironment env)
       throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
     try {
       updateTimestamp();
@@ -832,7 +775,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
   }
 
   @Override
-  public int compareTo(final Procedure<TEnvironment> other) {
+  public int compareTo(final Procedure other) {
     return Long.compare(getProcId(), other.getProcId());
   }
 
@@ -858,8 +801,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * Helper to lookup the root Procedure ID given a specified procedure.
    */
   @InterfaceAudience.Private
-  protected static Long getRootProcedureId(final Map<Long, Procedure> procedures,
-      Procedure<?> proc) {
+  protected static Long getRootProcedureId(final Map<Long, Procedure> procedures, Procedure proc) {
     while (proc.hasParent()) {
       proc = procedures.get(proc.getParentProcId());
       if (proc == null) return null;
@@ -872,10 +814,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
    * @param b the second procedure to be compared.
    * @return true if the two procedures have the same parent
    */
-  public static boolean haveSameParent(final Procedure<?> a, final Procedure<?> b) {
+  public static boolean haveSameParent(final Procedure a, final Procedure b) {
     if (a.hasParent() && b.hasParent()) {
       return a.getParentProcId() == b.getParentProcId();
     }
     return false;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 adb27a8..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
@@ -50,6 +50,6 @@ public class ProcedureEvent<T> {
   @Override
   public String toString() {
     return getClass().getSimpleName() + " for " + object + ", ready=" + isReady() +
-        ", " + getSuspendedProcedures();
+        ", suspended procedures count=" + getSuspendedProcedures().size();
   }
 }
\ No newline at end of file


[19/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
index 4d5953c..454e3bc 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/SnapshotProtos.java
@@ -357,7 +357,7 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
             (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -373,7 +373,7 @@ public final class SnapshotProtos {
         getNameBytes() {
       java.lang.Object ref = name_;
       if (ref instanceof java.lang.String) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
             org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         name_ = b;
@@ -407,7 +407,7 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
             (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -427,7 +427,7 @@ public final class SnapshotProtos {
         getTableBytes() {
       java.lang.Object ref = table_;
       if (ref instanceof java.lang.String) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
             org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         table_ = b;
@@ -499,7 +499,7 @@ public final class SnapshotProtos {
       if (ref instanceof java.lang.String) {
         return (java.lang.String) ref;
       } else {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
             (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
         java.lang.String s = bs.toStringUtf8();
         if (bs.isValidUtf8()) {
@@ -515,7 +515,7 @@ public final class SnapshotProtos {
         getOwnerBytes() {
       java.lang.Object ref = owner_;
       if (ref instanceof java.lang.String) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
             org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                 (java.lang.String) ref);
         owner_ = b;
@@ -1047,7 +1047,7 @@ public final class SnapshotProtos {
           getNameBytes() {
         java.lang.Object ref = name_;
         if (ref instanceof String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
               org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           name_ = b;
@@ -1135,7 +1135,7 @@ public final class SnapshotProtos {
           getTableBytes() {
         java.lang.Object ref = table_;
         if (ref instanceof String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
               org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           table_ = b;
@@ -1323,7 +1323,7 @@ public final class SnapshotProtos {
           getOwnerBytes() {
         java.lang.Object ref = owner_;
         if (ref instanceof String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b =
               org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
                   (java.lang.String) ref);
           owner_ = b;
@@ -1473,7 +1473,7 @@ public final class SnapshotProtos {
        * <code>optional .hbase.pb.UsersAndPermissions users_and_permissions = 7;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissionsOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissions.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.UsersAndPermissionsOrBuilder>
           getUsersAndPermissionsFieldBuilder() {
         if (usersAndPermissionsBuilder_ == null) {
           usersAndPermissionsBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -6366,7 +6366,7 @@ public final class SnapshotProtos {
 
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
     internal_static_hbase_pb_SnapshotDescription_descriptor;
-  private static final 
+  private static final
     org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
       internal_static_hbase_pb_SnapshotDescription_fieldAccessorTable;
   private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 fe95fd5..39e73b6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -39,10 +39,6 @@ message GetRegionInfoResponse {
   required RegionInfo region_info = 1;
   optional CompactionState compaction_state = 2;
   optional bool isRecovering = 3;
-  // True if region is splittable, false otherwise.
-  optional bool splittable = 4;
-  // True if region is mergeable, false otherwise.
-  optional bool mergeable = 5;
 
   enum CompactionState {
     NONE = 0;
@@ -124,6 +120,18 @@ 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,32 +268,6 @@ message ClearCompactionQueuesRequest {
 message ClearCompactionQueuesResponse {
 }
 
-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);
@@ -305,6 +287,9 @@ service AdminService {
   rpc CloseRegion(CloseRegionRequest)
     returns(CloseRegionResponse);
 
+  rpc CloseRegionForSplitOrMerge(CloseRegionForSplitOrMergeRequest)
+    returns(CloseRegionForSplitOrMergeResponse);
+
   rpc FlushRegion(FlushRegionRequest)
     returns(FlushRegionResponse);
 
@@ -344,10 +329,4 @@ service AdminService {
   /** Fetches the RegionServer's view of space quotas */
   rpc GetSpaceQuotaSnapshots(GetSpaceQuotaSnapshotsRequest)
     returns(GetSpaceQuotaSnapshotsResponse);
-
-  rpc ExecuteProcedures(ExecuteProceduresRequest)
-    returns(ExecuteProceduresResponse);
-
-  rpc MergeRegions(MergeRegionsRequest)
-    returns(MergeRegionsResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 7015fcb..3d6ae1b 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -81,21 +81,6 @@ 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.
  */
@@ -134,17 +119,6 @@ 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;
@@ -366,7 +340,6 @@ message RunCatalogScanRequest {
 }
 
 message RunCatalogScanResponse {
-  // This is how many archiving tasks we started as a result of this scan.
   optional int32 scan_result = 1;
 }
 
@@ -667,10 +640,6 @@ 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);
@@ -701,12 +670,6 @@ 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/a3c5a744/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 14d6d56..2c70882 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -265,31 +265,38 @@ message RestoreSnapshotStateData {
   repeated RestoreParentToChildRegionsPair parent_to_child_regions_pair_list = 7;
 }
 
-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;
+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;
 }
 
-message DispatchMergingRegionsStateData {
+message MergeTableRegionsStateData {
   required UserInformation user_info = 1;
-  required TableName table_name = 2;
-  repeated RegionInfo region_info = 3;
-  optional bool forcible = 4;
+  repeated RegionInfo region_info = 2;
+  required RegionInfo merged_region_info = 3;
+  optional bool forcible = 4 [default = false];
 }
 
 enum SplitTableRegionState {
   SPLIT_TABLE_REGION_PREPARE = 1;
   SPLIT_TABLE_REGION_PRE_OPERATION = 2;
-  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;
+  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;
 }
 
 message SplitTableRegionStateData {
@@ -298,29 +305,6 @@ 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;
@@ -342,56 +326,3 @@ 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 {
-  optional RegionInfo region_info = 1;
-  required ServerName source_server = 2;
-  required ServerName destination_server = 3;
-}
-
-enum GCRegionState {
-  GC_REGION_PREPARE = 1;
-  GC_REGION_ARCHIVE = 2;
-  GC_REGION_PURGE_METADATA = 3;
-}
-
-message GCRegionStateData {
-  required RegionInfo region_info = 1;
-}
-
-enum GCMergedRegionsState {
-  GC_MERGED_REGIONS_PREPARE = 1;
-  GC_MERGED_REGIONS_PURGE = 2;
-  GC_REGION_EDIT_METADATA = 3;
-}
-
-message GCMergedRegionsStateData {
-  required RegionInfo parent_a = 1;
-  required RegionInfo parent_b = 2;
-  required RegionInfo merged_child = 3;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 7d35df0..0b765d6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -26,6 +26,7 @@ option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
 import "HBase.proto";
+import "Master.proto";
 import "ClusterStatus.proto";
 
 message RegionServerStartupRequest {
@@ -126,6 +127,19 @@ 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;
+}
 
 message RegionSpaceUse {
   optional RegionInfo region_info = 1; // A region identifier
@@ -174,6 +188,18 @@ service RegionServerStatusService {
     returns(ReportRegionStateTransitionResponse);
 
   /**
+   * Split region
+   */
+  rpc SplitRegion(SplitTableRegionRequest)
+    returns(SplitTableRegionResponse);
+
+  /**
+   * Get procedure result
+   */
+  rpc getProcedureResult(GetProcedureResultRequest)
+    returns(GetProcedureResultResponse);
+
+  /**
    * Reports Region filesystem space use
    */
   rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest)

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 430c8fc..718c7c1 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,9 +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.HMaster;
-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;
@@ -119,14 +118,14 @@ public class RSGroupAdminServer implements RSGroupAdmin {
     LinkedList<HRegionInfo> regions = new LinkedList<>();
     for (Map.Entry<HRegionInfo, ServerName> el :
         master.getAssignmentManager().getRegionStates().getRegionAssignments().entrySet()) {
-      if (el.getValue() == null) continue;
       if (el.getValue().getAddress().equals(server)) {
         addRegion(regions, el.getKey());
       }
     }
-    for (RegionStateNode state : master.getAssignmentManager().getRegionsInTransition()) {
-      if (state.getRegionLocation().getAddress().equals(server)) {
-        addRegion(regions, state.getRegionInfo());
+    for (RegionState state:
+        this.master.getAssignmentManager().getRegionStates().getRegionsInTransition()) {
+      if (state.getServerName().getAddress().equals(server)) {
+        addRegion(regions, state.getRegion());
       }
     }
     return regions;
@@ -535,7 +534,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.moveAsync(plan);
+          assignmentManager.balance(plan);
         }
         LOG.info("RSGroup balance " + groupName + " completed after " +
             (System.currentTimeMillis()-startTime) + " seconds");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 e2dd91c..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
@@ -318,8 +318,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
   }
 
   private Map<ServerName, List<HRegionInfo>> correctAssignments(
-       Map<ServerName, List<HRegionInfo>> existingAssignments)
-  throws HBaseIOException{
+       Map<ServerName, List<HRegionInfo>> existingAssignments){
     Map<ServerName, List<HRegionInfo>> correctAssignments = new TreeMap<>();
     List<HRegionInfo> misplacedRegions = new LinkedList<>();
     correctAssignments.put(LoadBalancer.BOGUS_SERVER_NAME, new LinkedList<>());
@@ -347,11 +346,7 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
     //TODO bulk unassign?
     //unassign misplaced regions, so that they are assigned to correct groups.
     for(HRegionInfo info: misplacedRegions) {
-      try {
-        this.masterServices.getAssignmentManager().unassign(info);
-      } catch (IOException e) {
-        throw new HBaseIOException(e);
-      }
+      this.masterServices.getAssignmentManager().unassign(info);
     }
     return correctAssignments;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 0f1e849..83fe122 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.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.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/a3c5a744/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index 264ea39..ae1485c 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -51,13 +51,11 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Sets;
 
-@Ignore // TODO: Fix after HBASE-14614 goes in.
 @Category({MediumTests.class})
 public class TestRSGroups extends TestRSGroupsBase {
   protected static final Log LOG = LogFactory.getLog(TestRSGroups.class);
@@ -149,7 +147,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     });
   }
 
-  @Ignore @Test
+  @Test
   public void testBasicStartUp() throws IOException {
     RSGroupInfo defaultInfo = rsGroupAdmin.getRSGroupInfo(RSGroupInfo.DEFAULT_GROUP);
     assertEquals(4, defaultInfo.getServers().size());
@@ -159,7 +157,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     assertEquals(3, count);
   }
 
-  @Ignore @Test
+  @Test
   public void testNamespaceCreateAndAssign() throws Exception {
     LOG.info("testNamespaceCreateAndAssign");
     String nsName = tablePrefix+"_foo";
@@ -185,7 +183,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     Assert.assertEquals(1, ProtobufUtil.getOnlineRegions(rs).size());
   }
 
-  @Ignore @Test
+  @Test
   public void testDefaultNamespaceCreateAndAssign() throws Exception {
     LOG.info("testDefaultNamespaceCreateAndAssign");
     final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
@@ -203,7 +201,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     });
   }
 
-  @Ignore @Test
+  @Test
   public void testNamespaceConstraint() throws Exception {
     String nsName = tablePrefix+"_foo";
     String groupName = tablePrefix+"_foo";
@@ -238,7 +236,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     }
   }
 
-  @Ignore @Test
+  @Test
   public void testGroupInfoMultiAccessing() throws Exception {
     RSGroupInfoManager manager = rsGroupAdminEndpoint.getGroupInfoManager();
     RSGroupInfo defaultGroup = manager.getRSGroup("default");
@@ -249,7 +247,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     it.next();
   }
 
-  @Ignore @Test
+  @Test
   public void testMisplacedRegions() throws Exception {
     final TableName tableName = TableName.valueOf(tablePrefix+"_testMisplacedRegions");
     LOG.info("testMisplacedRegions");
@@ -277,7 +275,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     });
   }
 
-  @Ignore @Test
+  @Test
   public void testCloneSnapshot() throws Exception {
     byte[] FAMILY = Bytes.toBytes("test");
     String snapshotName = tableName.getNameAsString() + "_snap";

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
index 8b200ab..4802ca4 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -99,7 +98,7 @@ public class TestRSGroupsOfflineMode {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Ignore @Test
+  @Test
   public void testOffline() throws Exception, InterruptedException {
     // Table should be after group table name so it gets assigned later.
     final TableName failoverTable = TableName.valueOf(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 b5e6dd0..76a85a9 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,9 +18,7 @@ limitations under the License.
 </%doc>
 <%import>
 org.apache.hadoop.hbase.HRegionInfo;
-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.AssignmentManager;
 org.apache.hadoop.hbase.master.RegionState;
 org.apache.hadoop.conf.Configuration;
 org.apache.hadoop.hbase.HBaseConfiguration;
@@ -37,12 +35,28 @@ 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();
-RegionInTransitionStat ritStat = assignmentManager.computeRegionInTransitionStat();
+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());
+  }
+}
 
 int numOfRITs = rit.size();
 int ritsPerPage = Math.min(5, numOfRITs);
@@ -51,15 +65,15 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
     <section>
     <h2>Regions in Transition</h2>
      <p><% numOfRITs %> region(s) in transition.
-     <%if ritStat.hasRegionsTwiceOverThreshold()  %>
+     <%if !ritsTwiceThreshold.isEmpty()  %>
          <span class="label label-danger" style="font-size:100%;font-weight:normal">
-     <%elseif ritStat.hasRegionsOverThreshold() %>
+     <%elseif !ritsOverThreshold.isEmpty() %>
          <span class="label label-warning" style="font-size:100%;font-weight:normal">
      <%else>
          <span>
      </%if>
-         <% ritStat.getTotalRITsOverThreshold() %> region(s) in transition for
-             more than <% ritStat.getRITThreshold() %> milliseconds.
+         <% numOfRITOverThreshold %> region(s) in transition for
+             more than <% ritThreshold %> milliseconds.
          </span>
      </p>
      <div class="tabbable">
@@ -76,26 +90,25 @@ int numOfPages = (int) Math.ceil(numOfRITs * 1.0 / ritsPerPage);
                      <th>State</th><th>RIT time (ms)</th> <th>Retries </th></tr>
              </%if>
 
-             <%if ritStat.isRegionTwiceOverThreshold(rs.getRegion()) %>
-                     <tr class="alert alert-danger" role="alert">
-             <%elseif ritStat.isRegionOverThreshold(rs.getRegion()) %>
+             <%if ritsOverThreshold.contains(rs.getRegion().getEncodedName()) %>
                      <tr class="alert alert-warning" role="alert">
+             <%elseif ritsTwiceThreshold.contains(rs.getRegion().getEncodedName()) %>
+                     <tr class="alert alert-danger" role="alert">
             <%else>
                     <tr>
             </%if>
                         <%java>
                           String retryStatus = "0";
-                          RegionFailedOpen regionFailedOpen = assignmentManager
-                            .getRegionStates().getFailedOpen(rs.getRegion());
-                          if (regionFailedOpen != null) {
-                            retryStatus = Integer.toString(regionFailedOpen.getRetries());
+                          AtomicInteger numOpenRetries = failedRegionTracker.get(
+                              rs.getRegion().getEncodedName());
+                          if (numOpenRetries != null ) {
+                            retryStatus = Integer.toString(numOpenRetries.get());
                           } else if (rs.getState() ==  RegionState.State.FAILED_OPEN) {
-                            retryStatus = "Failed";
+                             retryStatus = "Failed";
                           }
                         </%java>
                         <td><% rs.getRegion().getEncodedName() %></td><td>
-                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs,
-                            assignmentManager.getConfiguration()) %></td>
+                        <% HRegionInfo.getDescriptiveNameFromRegionStateForDisplay(rs, conf) %></td>
                         <td><% (currentTime - rs.getStamp()) %> </td>
                         <td> <% retryStatus %> </td>
                      </tr>

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 14dfe0a..e1a47c5 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.assignment.AssignmentManager;
+org.apache.hadoop.hbase.master.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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
index 011ed1c..22725ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/RegionStateListener.java
@@ -26,8 +26,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public interface RegionStateListener {
-// TODO: Get rid of this!!!! Ain't there a better way to watch region
-// state than introduce a whole new listening mechanism? St.Ack
+
   /**
    * Process region split event.
    *
@@ -46,7 +45,9 @@ public interface RegionStateListener {
 
   /**
    * Process region merge event.
+   *
+   * @param hri An instance of HRegionInfo
    * @throws IOException
    */
-  void onRegionMerged(HRegionInfo mergedRegion) throws IOException;
+  void onRegionMerged(HRegionInfo hri) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
index 3fef686..3ecaa86 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/SplitLogTask.java
@@ -46,10 +46,6 @@ public class SplitLogTask {
   }
 
   public static class Owned extends SplitLogTask {
-    public Owned(final ServerName originServer) {
-      this(originServer, ZooKeeperProtos.SplitLogTask.RecoveryMode.LOG_SPLITTING);
-    }
- 
     public Owned(final ServerName originServer, final RecoveryMode mode) {
       super(originServer, ZooKeeperProtos.SplitLogTask.State.OWNED, mode);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index b9f52b8..ecd4401 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -73,16 +74,6 @@ public class HFileArchiver {
   }
 
   /**
-   * @return True if the Region exits in the filesystem.
-   */
-  public static boolean exists(Configuration conf, FileSystem fs, HRegionInfo info)
-      throws IOException {
-    Path rootDir = FSUtils.getRootDir(conf);
-    Path regionDir = HRegion.getRegionDir(rootDir, info);
-    return fs.exists(regionDir);
-  }
-
-  /**
    * Cleans up all the files for a HRegion by archiving the HFiles to the
    * archive directory
    * @param conf the configuration to use
@@ -146,7 +137,7 @@ public class HFileArchiver {
     FileStatus[] storeDirs = FSUtils.listStatus(fs, regionDir, nonHidden);
     // if there no files, we can just delete the directory and return;
     if (storeDirs == null) {
-      LOG.debug("Region directory " + regionDir + " empty.");
+      LOG.debug("Region directory (" + regionDir + ") was empty, just deleting and returning!");
       return deleteRegionWithoutArchiving(fs, regionDir);
     }
 
@@ -463,7 +454,7 @@ public class HFileArchiver {
   private static boolean deleteRegionWithoutArchiving(FileSystem fs, Path regionDir)
       throws IOException {
     if (fs.delete(regionDir, true)) {
-      LOG.debug("Deleted " + regionDir);
+      LOG.debug("Deleted all region files in: " + regionDir);
       return true;
     }
     LOG.debug("Failed to delete region directory:" + regionDir);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 4f134c0..ed1ae31 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,7 +35,9 @@ public final class VersionInfoUtil {
   }
 
   public static boolean currentClientHasMinimumVersion(int major, int minor) {
-    return hasMinimumVersion(getCurrentClientVersionInfo(), major, minor);
+    RpcCallContext call = RpcServer.getCurrentCall();
+    HBaseProtos.VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
+    return hasMinimumVersion(versionInfo, major, minor);
   }
 
   public static boolean hasMinimumVersion(HBaseProtos.VersionInfo versionInfo,
@@ -51,7 +53,7 @@ public final class VersionInfoUtil {
         return clientMinor >= minor;
       }
       try {
-        final String[] components = getVersionComponents(versionInfo);
+        String[] components = versionInfo.getVersion().split("\\.");
 
         int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
         if (clientMajor != major) {
@@ -66,79 +68,4 @@ 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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index ca68de2..e36feea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -448,8 +448,8 @@ public interface RegionObserver extends Coprocessor {
    * Called before the region is split.
    * @param c the environment provided by the region server
    * (e.getRegion() returns the parent region)
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
+   * @deprecated Use preSplit(
+   *    final ObserverContext&lt;RegionCoprocessorEnvironment&gt; c, byte[] splitRow)
    */
   @Deprecated
   default void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {}
@@ -460,8 +460,6 @@ public interface RegionObserver extends Coprocessor {
    * (e.getRegion() returns the parent region)
    *
    * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
    */
   @Deprecated
   default void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
@@ -473,8 +471,7 @@ public interface RegionObserver extends Coprocessor {
    * (e.getRegion() returns the parent region)
    * @param l the left daughter region
    * @param r the right daughter region
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
+   * @deprecated Use postCompleteSplit() instead
    */
   @Deprecated
   default void postSplit(final ObserverContext<RegionCoprocessorEnvironment> c, final Region l,
@@ -488,8 +485,6 @@ public interface RegionObserver extends Coprocessor {
    * @param metaEntries
    *
    * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
   */
   @Deprecated
   default void preSplitBeforePONR(final ObserverContext<RegionCoprocessorEnvironment> ctx,
@@ -500,9 +495,8 @@ public interface RegionObserver extends Coprocessor {
    * Calling {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} has no
    * effect in this hook.
    * @param ctx
+   *
    * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
   */
   @Deprecated
   default void preSplitAfterPONR(final ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -513,8 +507,6 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx
    *
    * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
   */
   @Deprecated
   default void preRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -525,8 +517,6 @@ public interface RegionObserver extends Coprocessor {
    * @param ctx
    *
    * Note: the logic moves to Master; it is unused in RS
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * @see MasterObserver
   */
   @Deprecated
   default void postRollBackSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -536,11 +526,7 @@ public interface RegionObserver extends Coprocessor {
    * Called after any split request is processed.  This will be called irrespective of success or
    * failure of the split.
    * @param ctx
-   * @deprecated No longer called in hbase2/AMv2 given the master runs splits now;
-   * implement {@link MasterObserver#postCompletedSplitRegionAction(ObserverContext, HRegionInfo, HRegionInfo)}
-   * instead.
    */
-  @Deprecated
   default void postCompleteSplit(final ObserverContext<RegionCoprocessorEnvironment> ctx)
     throws IOException {}
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 578fb0f..f476b11 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -135,14 +135,7 @@ public class CallRunner {
         RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call);
         return;
       } catch (Throwable e) {
-        if (e instanceof ServerNotRunningYetException) {
-          // If ServerNotRunningYetException, don't spew stack trace.
-          if (RpcServer.LOG.isTraceEnabled()) {
-            RpcServer.LOG.trace(call.toShortString(), e);
-          }
-        } else {
-          RpcServer.LOG.debug(call.toShortString(), e);
-        }
+        RpcServer.LOG.debug(Thread.currentThread().getName() + ": " + call.toShortString(), e);
         errorThrowable = e;
         error = StringUtils.stringifyException(e);
         if (e instanceof Error) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index 313535d..3cb6011 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -142,7 +142,7 @@ public abstract class RpcExecutor {
       queueClass = LinkedBlockingQueue.class;
     }
 
-    LOG.info("RpcExecutor " + name + " using " + callQueueType
+    LOG.info("RpcExecutor " + " name " + " using " + callQueueType
         + " as call queue; numCallQueues=" + numCallQueues + "; maxQueueLength=" + maxQueueLength
         + "; handlerCount=" + handlerCount);
   }
@@ -205,8 +205,6 @@ public abstract class RpcExecutor {
     double handlerFailureThreshhold = conf == null ? 1.0 : conf.getDouble(
       HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
       HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
-    LOG.debug("Started " + handlers.size() + " " + threadPrefix +
-        " handlers, qsize=" + qsize + " on port=" + port);
     for (int i = 0; i < numHandlers; i++) {
       final int index = qindex + (i % qsize);
       String name = "RpcServer." + threadPrefix + ".handler=" + handlers.size() + ",queue=" + index
@@ -214,6 +212,7 @@ public abstract class RpcExecutor {
       Handler handler = getHandler(name, handlerFailureThreshhold, callQueues.get(index),
         activeHandlerCount);
       handler.start();
+      LOG.debug("Started " + name);
       handlers.add(handler);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 040209b..481b701 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
@@ -130,7 +130,7 @@ public class SimpleRpcServer extends RpcServer {
       // has an advantage in that it is easy to shutdown the pool.
       readPool = Executors.newFixedThreadPool(readThreads,
         new ThreadFactoryBuilder().setNameFormat(
-          "Reader=%d,bindAddress=" + bindAddress.getHostName() +
+          "RpcServer.reader=%d,bindAddress=" + bindAddress.getHostName() +
           ",port=" + port).setDaemon(true)
         .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
       for (int i = 0; i < readThreads; ++i) {
@@ -142,7 +142,7 @@ public class SimpleRpcServer extends RpcServer {
 
       // Register accepts on the server socket with the selector.
       acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
-      this.setName("Listener,port=" + port);
+      this.setName("RpcServer.listener,port=" + port);
       this.setDaemon(true);
     }
 
@@ -331,7 +331,7 @@ public class SimpleRpcServer extends RpcServer {
         throw ieo;
       } catch (Exception e) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Caught exception while reading:", e);
+          LOG.debug(getName() + ": Caught exception while reading:", e);
         }
         count = -1; //so that the (count < 0) block is executed
       }
@@ -608,8 +608,8 @@ public class SimpleRpcServer extends RpcServer {
     SimpleServerRpcConnection register(SocketChannel channel) {
       SimpleServerRpcConnection connection = getConnection(channel, System.currentTimeMillis());
       add(connection);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Connection from " + connection +
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Server connection from " + connection +
             "; connections=" + size() +
             ", queued calls size (bytes)=" + callQueueSizeInBytes.sum() +
             ", general queued calls=" + scheduler.getGeneralQueueLength() +
@@ -621,8 +621,8 @@ public class SimpleRpcServer extends RpcServer {
     boolean close(SimpleServerRpcConnection connection) {
       boolean exists = remove(connection);
       if (exists) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(Thread.currentThread().getName() +
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(Thread.currentThread().getName() +
               ": disconnecting client " + connection +
               ". Number of active connections: "+ size());
         }
@@ -698,4 +698,4 @@ public class SimpleRpcServer extends RpcServer {
     }
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..4513a5d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
@@ -0,0 +1,49 @@
+/**
+ *
+ * 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;
+  }
+}


[07/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index c318ffc..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
+++ /dev/null
@@ -1,234 +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.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/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index aef67bf..66c5abf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.client;
 
 import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.ClientTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
index 10f466d..852c5cf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -17,24 +17,19 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-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.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -42,9 +37,17 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import java.io.IOException;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 @Category({MediumTests.class, ClientTests.class})
 public class TestSplitOrMergeStatus {
 
+  private static final Log LOG = LogFactory.getLog(TestSplitOrMergeStatus.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
 
@@ -74,7 +77,7 @@ public class TestSplitOrMergeStatus {
     TEST_UTIL.loadTable(t, FAMILY, false);
 
     RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
-    int originalCount = locator.getAllRegionLocations().size();
+    int orignalCount = locator.getAllRegionLocations().size();
 
     Admin admin = TEST_UTIL.getAdmin();
     initSwitchStatus(admin);
@@ -82,17 +85,14 @@ public class TestSplitOrMergeStatus {
     assertEquals(results.length, 1);
     assertTrue(results[0]);
     admin.split(t.getName());
-    int count = admin.getTableRegions(tableName).size();
-    assertTrue(originalCount == count);
+    int count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount == count);
     results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.SPLIT);
     assertEquals(results.length, 1);
     assertFalse(results[0]);
     admin.split(t.getName());
-    while ((count = admin.getTableRegions(tableName).size()) == originalCount) {
-      Threads.sleep(1);;
-    }
-    count = admin.getTableRegions(tableName).size();
-    assertTrue(originalCount < count);
+    count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount<count);
     admin.close();
   }
 
@@ -103,43 +103,33 @@ public class TestSplitOrMergeStatus {
     Table t = TEST_UTIL.createTable(tableName, FAMILY);
     TEST_UTIL.loadTable(t, FAMILY, false);
 
+    RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName());
+
     Admin admin = TEST_UTIL.getAdmin();
-    int originalCount = admin.getTableRegions(tableName).size();
     initSwitchStatus(admin);
     admin.split(t.getName());
-    int postSplitCount = -1;
-    while ((postSplitCount = admin.getTableRegions(tableName).size()) == originalCount) {
-      Threads.sleep(1);;
-    }
-    assertTrue("originalCount=" + originalCount + ", newCount=" + postSplitCount,
-        originalCount != postSplitCount);
+    waitOnSplitOrMerge(t); //Split the table to ensure we have two regions at least.
 
-    // Merge switch is off so merge should NOT succeed.
+    waitForMergable(admin, tableName);
+    int orignalCount = locator.getAllRegionLocations().size();
     boolean[] results = admin.setSplitOrMergeEnabled(false, false, MasterSwitchType.MERGE);
     assertEquals(results.length, 1);
     assertTrue(results[0]);
     List<HRegionInfo> regions = admin.getTableRegions(t.getName());
     assertTrue(regions.size() > 1);
-    Future<?> f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
+    admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
-    try {
-      f.get(10, TimeUnit.SECONDS);
-      fail("Should not get here.");
-    } catch (ExecutionException ee) {
-      // Expected.
-    }
-    int count = admin.getTableRegions(tableName).size();
-    assertTrue("newCount=" + postSplitCount + ", count=" + count, postSplitCount == count);
+    int count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount == count);
 
+    waitForMergable(admin, tableName);
     results = admin.setSplitOrMergeEnabled(true, false, MasterSwitchType.MERGE);
-    regions = admin.getTableRegions(t.getName());
     assertEquals(results.length, 1);
     assertFalse(results[0]);
-    f = admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
+    admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
       regions.get(1).getEncodedNameAsBytes(), true);
-    f.get(10, TimeUnit.SECONDS);
-    count = admin.getTableRegions(tableName).size();
-    assertTrue((postSplitCount / 2 /*Merge*/) == count);
+    count = waitOnSplitOrMerge(t).size();
+    assertTrue(orignalCount>count);
     admin.close();
   }
 
@@ -166,4 +156,47 @@ public class TestSplitOrMergeStatus {
     assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
     assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
   }
-}
\ No newline at end of file
+
+  private void waitForMergable(Admin admin, TableName t) throws InterruptedException, IOException {
+    // Wait for the Regions to be mergeable
+    MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
+    int mergeable = 0;
+    while (mergeable < 2) {
+      Thread.sleep(100);
+      admin.majorCompact(t);
+      mergeable = 0;
+      for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
+        for (Region region: regionThread.getRegionServer().getOnlineRegions(t)) {
+          mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
+        }
+      }
+    }
+  }
+
+  /*
+   * Wait on table split.  May return because we waited long enough on the split
+   * and it didn't happen.  Caller should check.
+   * @param t
+   * @return Map of table regions; caller needs to check table actually split.
+   */
+  private List<HRegionLocation> waitOnSplitOrMerge(final Table t)
+    throws IOException {
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
+      List<HRegionLocation> regions = locator.getAllRegionLocations();
+      int originalCount = regions.size();
+      for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 10); i++) {
+        Thread.currentThread();
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+        regions = locator.getAllRegionLocations();
+        if (regions.size() !=  originalCount)
+          break;
+      }
+      return regions;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8412e77..3eb65a5 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
@@ -35,28 +35,29 @@ 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.NamespaceDescriptor;
+import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 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.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.LoadOnlyFavoredStochasticBalancer;
-import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.util.Threads;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -65,9 +66,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
 
 @Category({ClientTests.class, MediumTests.class})
 public class TestTableFavoredNodes {
@@ -78,6 +76,7 @@ public class TestTableFavoredNodes {
   private final static int WAIT_TIMEOUT = 60000;
   private final static int SLAVES = 8;
   private FavoredNodesManager fnm;
+  private RegionStates regionStates;
   private Admin admin;
 
   private final byte[][] splitKeys = new byte[][] {Bytes.toBytes(1), Bytes.toBytes(9)};
@@ -102,8 +101,8 @@ public class TestTableFavoredNodes {
 
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
     TEST_UTIL.cleanupTestDir();
+    TEST_UTIL.shutdownMiniCluster();
   }
 
   @Before
@@ -112,6 +111,8 @@ public class TestTableFavoredNodes {
     admin = TEST_UTIL.getAdmin();
     admin.setBalancerRunning(false, true);
     admin.enableCatalogJanitor(false);
+    regionStates =
+      TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
   }
 
   /*
@@ -164,9 +165,8 @@ public class TestTableFavoredNodes {
   @Test
   public void testSplitTable() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    Table t = TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
+    TEST_UTIL.createTable(tableName, Bytes.toBytes("f"), splitKeys);
     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
-    final int numberOfRegions = admin.getTableRegions(t.getName()).size();
 
     checkIfFavoredNodeInformationIsCorrect(tableName);
 
@@ -176,14 +176,13 @@ public class TestTableFavoredNodes {
     List<ServerName> parentFN = fnm.getFavoredNodes(parent);
     assertNotNull("FN should not be null for region: " + parent, parentFN);
 
-    LOG.info("SPLITTING TABLE");
     admin.split(tableName, splitPoint);
 
     TEST_UTIL.waitUntilNoRegionsInTransition(WAIT_TIMEOUT);
-    LOG.info("FINISHED WAITING ON RIT");
-    waitUntilTableRegionCountReached(tableName, numberOfRegions + 1);
+    waitUntilTableRegionCountReached(tableName, NUM_REGIONS + 1);
 
-    // All regions should have favored nodes    checkIfFavoredNodeInformationIsCorrect(tableName);
+    // All regions should have favored nodes
+    checkIfFavoredNodeInformationIsCorrect(tableName);
 
     // Get the daughters of parent.
     HRegionInfo daughter1 = locator.getRegionLocation(parent.getStartKey(), true).getRegionInfo();
@@ -207,18 +206,11 @@ public class TestTableFavoredNodes {
 
     // Major compact table and run catalog janitor. Parent's FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
-    admin.runCatalogScan();
-    // Catalog cleanup is async. Wait on procedure to finish up.
-    ProcedureTestingUtility.waitAllProcedures(
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
-    // assertEquals("Parent region should have been cleaned", 1, admin.runCatalogScan());
+    assertEquals("Parent region should have been cleaned", 1, admin.runCatalogScan());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(parent));
 
     List<HRegionInfo> regions = admin.getTableRegions(tableName);
-    // Split and Table Disable interfere with each other around region replicas
-    // TODO. Meantime pause a few seconds.
-    Threads.sleep(2000);
-    LOG.info("STARTING DELETE");
+
     TEST_UTIL.deleteTable(tableName);
 
     checkNoFNForDeletedTable(regions);
@@ -243,12 +235,11 @@ 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, countOfRegions - 1);
+    waitUntilTableRegionCountReached(tableName, NUM_REGIONS - 1);
 
     // All regions should have favored nodes
     checkIfFavoredNodeInformationIsCorrect(tableName);
@@ -263,9 +254,6 @@ public class TestTableFavoredNodes {
     // Major compact table and run catalog janitor. Parent FN should be removed
     TEST_UTIL.getMiniHBaseCluster().compact(tableName, true);
     assertEquals("Merge parents should have been cleaned", 1, admin.runCatalogScan());
-    // Catalog cleanup is async. Wait on procedure to finish up.
-    ProcedureTestingUtility.waitAllProcedures(
-        TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor());
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionA));
     assertNull("Parent FN should be null", fnm.getFavoredNodes(regionB));
 
@@ -278,7 +266,6 @@ public class TestTableFavoredNodes {
 
   private void checkNoFNForDeletedTable(List<HRegionInfo> regions) {
     for (HRegionInfo region : regions) {
-      LOG.info("Testing if FN data for " + region);
       assertNull("FN not null for deleted table's region: " + region, fnm.getFavoredNodes(region));
     }
   }
@@ -389,8 +376,8 @@ public class TestTableFavoredNodes {
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return MetaTableAccessor.getRegionCount(TEST_UTIL.getConfiguration(), tableName) == numRegions;
+        return regionStates.getRegionsOfTable(tableName).size() == numRegions;
       }
     });
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8805337..35ed531 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,6 +32,7 @@ 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;
@@ -39,7 +40,6 @@ 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,9 +84,6 @@ 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/a3c5a744/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 a1b33f7..1d7c12e 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
@@ -49,7 +49,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 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.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Put;
@@ -67,19 +66,17 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestRule;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 @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);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
index b6ad2c9..7f1723c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Deque;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -59,7 +60,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 626e154..ff6b88e 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,17 +32,14 @@ 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.favored.FavoredNodesManager;
-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;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockInfo;
-import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -377,11 +374,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
-    return false;
-  }
-
-  @Override
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException {
   }
@@ -434,16 +426,4 @@ 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;
-  }
-
-  @Override
-  public ProcedureEvent getInitializedEvent() {
-    // TODO Auto-generated method stub
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 7ac7571..67a3f0a 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,6 +41,7 @@ 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;
@@ -53,10 +54,10 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompac
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearCompactionQueuesResponse;
 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;
@@ -69,8 +70,6 @@ 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;
@@ -503,6 +502,13 @@ 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
@@ -510,6 +516,16 @@ 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;
@@ -722,22 +738,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
   }
 
   @Override
-  public ExecuteProceduresResponse executeProcedures(RpcController controller,
-      ExecuteProceduresRequest request) throws ServiceException {
-    return null;
-  }
-
-  @Override
   public GetSpaceQuotaSnapshotsResponse getSpaceQuotaSnapshots(
       RpcController controller, GetSpaceQuotaSnapshotsRequest 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/a3c5a744/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 95c0615..78b75d5 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,7 +40,6 @@ 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;


[09/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index e7157d0..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplit.java
+++ /dev/null
@@ -1,723 +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.quotas.RegionServerSpaceQuotaManager;
-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();
-  }
-
-  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");
-    }
-
-    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
-    }
-
-    final RegionServerSpaceQuotaManager spaceQuotaManager =
-      this.server.getRegionServerSpaceQuotaManager();
-    if (spaceQuotaManager != null && spaceQuotaManager.areCompactionsDisabled(
-        r.getTableDesc().getTableName())) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
-            + " policy disallows compactions.");
-      }
-      return null;
-    }
-
-    // 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();
-    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(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;
-    private long time;
-
-    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;
-      this.time =  System.currentTimeMillis();
-    }
-
-    @Override
-    public String toString() {
-      return (this.compaction != null) ? ("Request = " + compaction.getRequest())
-          : ("regionName = " + region.toString() + ", storeName = " + store.toString() +
-             ", priority = " + queuedPriority + ", time = " + time);
-    }
-
-    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);
-      }
-    }
-
-    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();
-  }
-
-  /**
-   * {@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
-  /**
-   * 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();
-  }
-
-  public void clearLongCompactionsQueue() {
-    longCompactions.getQueue().clear();
-  }
-
-  public void clearShortCompactionsQueue() {
-    shortCompactions.getQueue().clear();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..7791ea7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -0,0 +1,695 @@
+/**
+ *
+ * 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.quotas.RegionServerSpaceQuotaManager;
+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;
+
+  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 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);
+          }
+      });
+
+    // 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();
+  }
+  
+  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");
+    }
+
+    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
+    }
+
+    final RegionServerSpaceQuotaManager spaceQuotaManager =
+      this.server.getRegionServerSpaceQuotaManager();
+    if (spaceQuotaManager != null && spaceQuotaManager.areCompactionsDisabled(
+        r.getTableDesc().getTableName())) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Ignoring compaction request for " + r + " as an active space quota violation "
+            + " policy disallows compactions.");
+      }
+      return null;
+    }
+
+    // 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();
+    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(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;
+    private long time;
+
+    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;
+      this.time =  System.currentTimeMillis();
+    }
+
+    @Override
+    public String toString() {
+      return (this.compaction != null) ? ("Request = " + compaction.getRequest())
+          : ("regionName = " + region.toString() + ", storeName = " + store.toString() +
+             ", priority = " + queuedPriority + ", time = " + time);
+    }
+
+    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);
+      }
+    }
+
+    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();
+  }
+
+  /**
+   * {@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
+  /**
+   * 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();
+  }
+
+  public void clearLongCompactionsQueue() {
+    longCompactions.getQueue().clear();
+  }
+
+  public void clearShortCompactionsQueue() {
+    shortCompactions.getQueue().clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
index 6b8948b..2773e00 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactedHFilesDischarger.java
@@ -34,8 +34,8 @@ import com.google.common.annotations.VisibleForTesting;
 
 /**
  * A chore service that periodically cleans up the compacted files when there are no active readers
- * using those compacted files and also helps in clearing the block cache of these compacted
- * file entries.
+ * using those compacted files and also helps in clearing the block cache with these compacted
+ * file entries
  */
 @InterfaceAudience.Private
 public class CompactedHFilesDischarger extends ScheduledChore {
@@ -71,56 +71,45 @@ public class CompactedHFilesDischarger extends ScheduledChore {
     this.useExecutor = useExecutor;
   }
 
-  /**
-   * CompactedHFilesDischarger runs asynchronously by default using the hosting
-   * RegionServer's Executor. In tests it can be useful to force a synchronous
-   * cleanup. Use this method to set no-executor before you call run.
-   * @return The old setting for <code>useExecutor</code>
-   */
-  @VisibleForTesting
-  boolean setUseExecutor(final boolean useExecutor) {
-    boolean oldSetting = this.useExecutor;
-    this.useExecutor = useExecutor;
-    return oldSetting;
-  }
-
   @Override
   public void chore() {
     // Noop if rss is null. This will never happen in a normal condition except for cases
     // when the test case is not spinning up a cluster
     if (regionServerServices == null) return;
     List<Region> onlineRegions = regionServerServices.getOnlineRegions();
-    if (onlineRegions == null) return;
-    for (Region region : onlineRegions) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo());
-      }
-      for (Store store : region.getStores()) {
-        try {
-          if (useExecutor && regionServerServices != null) {
-            CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
-                (Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER,
-                (HStore) store);
-            regionServerServices.getExecutorService().submit(handler);
-          } else {
-            // call synchronously if the RegionServerServices are not
-            // available
-            store.closeAndArchiveCompactedFiles();
-          }
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Completed archiving the compacted files for the region "
-                + region.getRegionInfo() + " under the store " + store.getColumnFamilyName());
+    if (onlineRegions != null) {
+      for (Region region : onlineRegions) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(
+              "Started the compacted hfiles cleaner for the region " + region.getRegionInfo());
+        }
+        for (Store store : region.getStores()) {
+          try {
+            if (useExecutor && regionServerServices != null) {
+              CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
+                  (Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER,
+                  (HStore) store);
+              regionServerServices.getExecutorService().submit(handler);
+            } else {
+              // call synchronously if the RegionServerServices are not
+              // available
+              store.closeAndArchiveCompactedFiles();
+            }
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Completed archiving the compacted files for the region "
+                  + region.getRegionInfo() + " under the store " + store.getColumnFamilyName());
+            }
+          } catch (Exception e) {
+            LOG.error("Exception while trying to close and archive the compacted store "
+                + "files of the store  " + store.getColumnFamilyName() + " in the" + " region "
+                + region.getRegionInfo(), e);
           }
-        } catch (Exception e) {
-          LOG.error("Exception while trying to close and archive the compacted store "
-              + "files of the store  " + store.getColumnFamilyName() + " in the" + " region "
-              + region.getRegionInfo(), e);
         }
-      }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(
-            "Completed the compacted hfiles cleaner for the region " + region.getRegionInfo());
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(
+              "Completed the compacted hfiles cleaner for the region " + region.getRegionInfo());
+        }
       }
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 8cc9cd7..a4a7537 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1390,14 +1390,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return !isClosed() && !isClosing();
   }
 
-  @Override
+  /** @return true if region is splittable */
   public boolean isSplittable() {
-    boolean result = isAvailable() && !hasReferences();
-    LOG.info("ASKED IF SPLITTABLE " + result, new Throwable("LOGGING"));
-    return result;
+    return isAvailable() && !hasReferences();
   }
 
-  @Override
+  /**
+   * @return true if region is mergeable
+   */
   public boolean isMergeable() {
     if (!isAvailable()) {
       LOG.debug("Region " + this
@@ -5086,15 +5086,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  @Override
-  public boolean refreshStoreFiles() throws IOException {
-    return refreshStoreFiles(false);
-  }
-
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
       justification = "Notify is about post replay. Intentional")
-  protected boolean refreshStoreFiles(boolean force) throws IOException {
-    if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
+  @Override
+  public boolean refreshStoreFiles() throws IOException {
+    if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
       return false; // if primary nothing to do
     }
 
@@ -5852,12 +5848,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       try {
         for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
           Store store = stores.get(entry.getKey());
-          KeyValueScanner scanner;
-          try {
-            scanner = store.getScanner(scan, entry.getValue(), this.readPt);
-          } catch (FileNotFoundException e) {
-            throw handleFileNotFound(e);
-          }
+          KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt);
           instantiatedScanners.add(scanner);
           if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
               || this.filter.isFamilyEssential(entry.getKey())) {
@@ -5881,20 +5872,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
     }
 
-    private FileNotFoundException handleFileNotFound(FileNotFoundException fnfe) {
+    private void handleFileNotFound(Throwable fnfe) {
       // Try reopening the region since we have lost some storefiles.
       // See HBASE-17712 for more details.
-      LOG.warn("Store file is lost; close and reopen region", fnfe);
+      LOG.warn("A store file got lost, so close and reopen region", fnfe);
       if (regionUnassigner != null) {
         regionUnassigner.unassign();
       }
-      return fnfe;
     }
 
     private IOException handleException(List<KeyValueScanner> instantiatedScanners,
         Throwable t) {
       if (t instanceof FileNotFoundException) {
-        handleFileNotFound((FileNotFoundException)t);
+        handleFileNotFound(t);
       }
       // remove scaner read point before throw the exception
       scannerReadPoints.remove(this);
@@ -6040,33 +6030,29 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       boolean tmpKeepProgress = scannerContext.getKeepProgress();
       // Scanning between column families and thus the scope is between cells
       LimitScope limitScope = LimitScope.BETWEEN_CELLS;
-      try {
-        do {
-          // We want to maintain any progress that is made towards the limits while scanning across
-          // different column families. To do this, we toggle the keep progress flag on during calls
-          // to the StoreScanner to ensure that any progress made thus far is not wiped away.
-          scannerContext.setKeepProgress(true);
-          heap.next(results, scannerContext);
-          scannerContext.setKeepProgress(tmpKeepProgress);
-
-          nextKv = heap.peek();
-          moreCellsInRow = moreCellsInRow(nextKv, currentRowCell);
-          if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
-          if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
-            return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
-          } else if (scannerContext.checkSizeLimit(limitScope)) {
-            ScannerContext.NextState state =
-                moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
-            return scannerContext.setScannerState(state).hasMoreValues();
-          } else if (scannerContext.checkTimeLimit(limitScope)) {
-            ScannerContext.NextState state =
-                moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
-            return scannerContext.setScannerState(state).hasMoreValues();
-          }
-        } while (moreCellsInRow);
-      } catch (FileNotFoundException e) {
-        throw handleFileNotFound(e);
-      }
+      do {
+        // We want to maintain any progress that is made towards the limits while scanning across
+        // different column families. To do this, we toggle the keep progress flag on during calls
+        // to the StoreScanner to ensure that any progress made thus far is not wiped away.
+        scannerContext.setKeepProgress(true);
+        heap.next(results, scannerContext);
+        scannerContext.setKeepProgress(tmpKeepProgress);
+
+        nextKv = heap.peek();
+        moreCellsInRow = moreCellsInRow(nextKv, currentRowCell);
+        if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
+        if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
+          return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
+        } else if (scannerContext.checkSizeLimit(limitScope)) {
+          ScannerContext.NextState state =
+              moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
+          return scannerContext.setScannerState(state).hasMoreValues();
+        } else if (scannerContext.checkTimeLimit(limitScope)) {
+          ScannerContext.NextState state =
+              moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
+          return scannerContext.setScannerState(state).hasMoreValues();
+        }
+      } while (moreCellsInRow);
       return nextKv != null;
     }
 
@@ -6415,7 +6401,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           result = this.joinedHeap.requestSeek(kv, true, true) || result;
         }
       } catch (FileNotFoundException e) {
-        throw handleFileNotFound(e);
+        handleFileNotFound(e);
+        throw e;
       } finally {
         closeRegionOperation();
       }
@@ -7800,11 +7787,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       return null;
     }
 
-    // Can't split a region that is closing.
-    if (this.isClosing()) {
-      return null;
-    }
-
     if (!splitPolicy.shouldSplit()) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 59a0fe5..014427d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -318,15 +318,13 @@ public class HRegionFileSystem {
    * @throws IOException
    */
   public boolean hasReferences(final String familyName) throws IOException {
-    Path storeDir = getStoreDir(familyName);
-    FileStatus[] files = FSUtils.listStatus(fs, storeDir);
+    FileStatus[] files = FSUtils.listStatus(fs, getStoreDir(familyName));
     if (files != null) {
       for(FileStatus stat: files) {
         if(stat.isDirectory()) {
           continue;
         }
         if(StoreFileInfo.isReference(stat.getPath())) {
-          if (LOG.isTraceEnabled()) LOG.trace("Reference " + stat.getPath());
           return true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 9315b0a..3ca061a 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
@@ -86,6 +86,7 @@ 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;
@@ -169,6 +170,8 @@ 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;
@@ -176,6 +179,7 @@ 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;
@@ -204,23 +208,21 @@ 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.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
-@SuppressWarnings({ "deprecation"})
+@SuppressWarnings({ "deprecation", "restriction" })
 public class HRegionServer extends HasThread implements
     RegionServerServices, LastSequenceId, ConfigurationObserver {
-  // Time to pause if master says 'please hold'. Make configurable if needed.
-  private static final int INIT_PAUSE_TIME_MS = 1000;
 
   public static final String REGION_LOCK_AWAIT_TIME_SEC =
       "hbase.regionserver.region.lock.await.time.sec";
@@ -281,7 +283,7 @@ public class HRegionServer extends HasThread implements
   protected ReplicationSinkService replicationSinkHandler;
 
   // Compactions
-  public CompactSplit compactSplitThread;
+  public CompactSplitThread compactSplitThread;
 
   /**
    * Map of regions currently being served by this region server. Key is the
@@ -512,8 +514,7 @@ public class HRegionServer extends HasThread implements
    */
   protected final ConfigurationManager configurationManager;
 
-  @VisibleForTesting
-  CompactedHFilesDischarger compactedFileDischarger;
+  private CompactedHFilesDischarger compactedFileDischarger;
 
   private volatile ThroughputController flushThroughputController;
 
@@ -913,7 +914,7 @@ public class HRegionServer extends HasThread implements
     this.cacheFlusher = new MemStoreFlusher(conf, this);
 
     // Compaction thread
-    this.compactSplitThread = new CompactSplit(this);
+    this.compactSplitThread = new CompactSplitThread(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.
@@ -1431,7 +1432,7 @@ public class HRegionServer extends HasThread implements
             // Only print out regions still closing if a small number else will
             // swamp the log.
             if (count < 10 && LOG.isDebugEnabled()) {
-              LOG.debug("Online Regions=" + this.onlineRegions);
+              LOG.debug(this.onlineRegions);
             }
           }
         }
@@ -1778,7 +1779,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("MemstoreFlusherChore", server, cacheFlushInterval);
+      super(server.getServerName() + "-MemstoreFlusherChore", server, cacheFlushInterval);
       this.server = server;
     }
 
@@ -2191,8 +2192,6 @@ public class HRegionServer extends HasThread implements
       transition.addRegionInfo(HRegionInfo.convert(hri));
     }
     ReportRegionStateTransitionRequest request = builder.build();
-    int tries = 0;
-    long pauseTime = INIT_PAUSE_TIME_MS;
     while (keepLooping()) {
       RegionServerStatusService.BlockingInterface rss = rssStub;
       try {
@@ -2203,40 +2202,95 @@ public class HRegionServer extends HasThread implements
         ReportRegionStateTransitionResponse response =
           rss.reportRegionStateTransition(null, request);
         if (response.hasErrorMessage()) {
-          LOG.info("Failed transition " + hris[0]
+          LOG.info("Failed to transition " + hris[0]
             + " to " + code + ": " + response.getErrorMessage());
           return false;
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("TRANSITION REPORTED " + request);
-        }
         return true;
       } catch (ServiceException se) {
         IOException ioe = ProtobufUtil.getRemoteException(se);
-        boolean pause = ioe instanceof ServerNotRunningYetException ||
-            ioe instanceof PleaseHoldException;
-        if (pause) {
-          // Do backoff else we flood the Master with requests.
-          pauseTime = ConnectionUtils.getPauseTime(pauseTime, tries);
-        } else {
-          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        LOG.info("Failed to report region transition, will retry", ioe);
+        if (rssStub == rss) {
+          rssStub = null;
+        }
+      }
+    }
+    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;
         }
-        LOG.info("Failed report of region transition; retry (#" + tries + ")" +
-            (pause?
-                " after " + pauseTime + "ms delay (Master is coming online...).":
-                " immediately."),
-            ioe);
-        if (pause) Threads.sleep(pauseTime);
-        tries++;
+
+        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;
         }
       }
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("TRANSITION NOT REPORTED " + request);
+    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 false;
+    return true;
   }
 
   /**
@@ -2927,7 +2981,7 @@ public class HRegionServer extends HasThread implements
    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
    */
   public static void main(String[] args) throws Exception {
-    LOG.info("STARTING service '" + HRegionServer.class.getSimpleName());
+    LOG.info("***** STARTING service '" + HRegionServer.class.getSimpleName() + "' *****");
     VersionInfo.logVersion();
     Configuration conf = HBaseConfiguration.create();
     @SuppressWarnings("unchecked")
@@ -3232,7 +3286,7 @@ public class HRegionServer extends HasThread implements
         throw new RegionOpeningException("Region " + regionNameStr +
           " is opening on " + this.serverName);
       }
-      throw new NotServingRegionException("" + regionNameStr +
+      throw new NotServingRegionException("Region " + regionNameStr +
         " is not online on " + this.serverName);
     }
     return region;
@@ -3350,7 +3404,7 @@ public class HRegionServer extends HasThread implements
   }
 
   // This map will contains all the regions that we closed for a move.
-  // We add the time it was moved as we don't want to keep too old information
+  //  We add the time it was moved as we don't want to keep too old information
   protected Map<String, MovedRegionInfo> movedRegions =
       new ConcurrentHashMap<>(3000);
 
@@ -3462,9 +3516,9 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return the underlying {@link CompactSplit} for the servers
+   * @return the underlying {@link CompactSplitThread} for the servers
    */
-  public CompactSplit getCompactSplitThread() {
+  public CompactSplitThread getCompactSplitThread() {
     return this.compactSplitThread;
   }
 


[15/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
deleted file mode 100644
index 864b7f1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ /dev/null
@@ -1,1709 +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.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 java.util.stream.Collectors;
-
-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.master.balancer.FavoredStochasticBalancer;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
-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.NoSuchProcedureException;
-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 from hbase1 to hbase2.
-  //  - handle sys table assignment first (e.g. acl, namespace)
-  //  - handle table priorities
-  //  - If ServerBusyException trying to update hbase:meta, we abort the Master
-  //   See updateRegionLocation in RegionStateStore.
-  //
-  // See also
-  // https://docs.google.com/document/d/1eVKa7FHdeoJ1-9o8yZcOTAQbv0u0bblBlCCzVSIn69g/edit#heading=h.ystjyrkbtoq5
-  // for other TODOs.
-
-  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 regionStates = 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 = FavoredStochasticBalancer.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
-    regionStates.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;
-  }
-
-  public RegionStates getRegionStates() {
-    return regionStates;
-  }
-
-  public RegionStateStore getRegionStateStore() {
-    return regionStateStore;
-  }
-
-  public List<ServerName> getFavoredNodes(final HRegionInfo regionInfo) {
-    return this.shouldAssignRegionsWithFavoredNodes?
-        ((FavoredStochasticBalancer)getBalancer()).getFavoredNodes(regionInfo):
-          ServerName.EMPTY_SERVER_LIST;
-  }
-
-  // ============================================================================================
-  //  Table State Manager helpers
-  // ============================================================================================
-  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 (isCarryingRegion(serverName, hri)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private boolean isCarryingRegion(final ServerName serverName, final HRegionInfo regionInfo) {
-    // TODO: check for state?
-    final RegionStateNode node = regionStates.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) throws IOException {
-    unassign(regionInfo, false);
-  }
-
-  public void unassign(final HRegionInfo regionInfo, final boolean forceNewPlan)
-  throws IOException {
-    // TODO: rename this reassign
-    RegionStateNode node = this.regionStates.getRegionNode(regionInfo);
-    ServerName destinationServer = node.getRegionLocation();
-    if (destinationServer == null) {
-      throw new UnexpectedStateException("DestinationServer is null; Assigned? " + node.toString());
-    }
-    assert destinationServer != null; node.toString();
-    UnassignProcedure proc = createUnassignProcedure(regionInfo, destinationServer, forceNewPlan);
-    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
-  }
-
-  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();
-    // Something badly wrong if takes ten seconds to register a region.
-    long endTime = startTime + 10000;
-    while ((node = regionStates.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) {
-      throw new NoSuchProcedureException(node.toString());
-    }
-
-    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) {
-    if (regionInfo.isEmpty()) return null;
-    final AssignProcedure[] procs = new AssignProcedure[regionInfo.size()];
-    int index = 0;
-    for (HRegionInfo hri: regionInfo) {
-      procs[index++] = createAssignProcedure(hri, forceNewPlan);
-    }
-    return procs;
-  }
-
-  // Needed for the following method so it can type the created Array we return
-  private static final UnassignProcedure [] UNASSIGNED_PROCEDURE_FOR_TYPE_INFO =
-      new UnassignProcedure[0];
-
-  UnassignProcedure[] createUnassignProcedures(final Collection<RegionStateNode> nodes) {
-    if (nodes.isEmpty()) return null;
-    final List<UnassignProcedure> procs = new ArrayList<UnassignProcedure>(nodes.size());
-    for (RegionStateNode node: nodes) {
-      if (!this.regionStates.include(node, false)) continue;
-      // Look for regions that are offline/closed; i.e. already unassigned.
-      if (this.regionStates.isRegionOffline(node.getRegionInfo())) continue;
-      assert node.getRegionLocation() != null: node.toString();
-      procs.add(createUnassignProcedure(node.getRegionInfo(), node.getRegionLocation(), false));
-    }
-    return procs.toArray(UNASSIGNED_PROCEDURE_FOR_TYPE_INFO);
-  }
-
-  public MoveRegionProcedure[] createReopenProcedures(final Collection<HRegionInfo> regionInfo) {
-    final MoveRegionProcedure[] procs = new MoveRegionProcedure[regionInfo.size()];
-    int index = 0;
-    for (HRegionInfo hri: regionInfo) {
-      final ServerName serverName = regionStates.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(regionStates.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(regionStates.getTableRegionStateNodes(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(regionStates.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) {
-    // If destinationServer is null, figure it.
-    ServerName sn = destinationServer != null? destinationServer:
-      getRegionStates().getRegionState(regionInfo).getServerName();
-    assert sn != null;
-    UnassignProcedure proc = new UnassignProcedure(regionInfo, sn, force);
-    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
-    return proc;
-  }
-
-  public MoveRegionProcedure createMoveRegionProcedure(final RegionPlan plan) {
-    return new MoveRegionProcedure(getProcedureEnvironment(), plan);
-  }
-
-
-  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 = regionStates.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
-      regionStates.removeFromOfflineRegions(regionInfo);
-      regionStates.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) {
-      if (LOG.isTraceEnabled()) LOG.trace("Failed 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 transitions failed.
-      LOG.warn("Failed transition", e);
-      builder.setErrorMessage("Failed 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 = regionStates.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 = regionStates.getOrCreateServer(serverName);
-    if (!reportTransition(regionNode, serverNode, state, seqId)) {
-      LOG.warn(String.format(
-        "No procedure for %s. server=%s to transition to %s", regionNode, serverName, state));
-    }
-  }
-
-  // FYI: regionNode is sometimes synchronized by the caller but not always.
-  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 {
-      if (regionStateListener != null) {
-        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("Split request from " + 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 (regionStates.getOrCreateServer(serverName).getVersionNumber() < 0x0200000) {
-      throw new UnsupportedOperationException(String.format(
-        "Split handled by the master: parent=%s hriA=%s hriB=%s", parent.getShortNameToLog(), 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 (regionStates.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() + " " +
-          regionNames.stream().map(element -> Bytes.toStringBinary(element)).
-            collect(Collectors.toList()));
-    }
-
-    final ServerStateNode serverNode = regionStates.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 = regionStates.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 " + serverName + ": " + e.getMessage());
-      killRegionServer(serverNode);
-    }
-  }
-
-  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 = regionStates.getRegionNodeFromName(regionName);
-        if (regionNode == null) {
-          throw new UnexpectedStateException("Not online: " + Bytes.toStringBinary(regionName));
-        }
-        synchronized (regionNode) {
-          if (regionNode.isInState(State.OPENING, State.OPEN)) {
-            if (!regionNode.getRegionLocation().equals(serverName)) {
-              throw new UnexpectedStateException(regionNode.toString() +
-                "reported OPEN on server=" + serverName +
-                " but state has otherwise.");
-            } else if (regionNode.isInState(State.OPENING)) {
-              try {
-                if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
-                  LOG.warn(regionNode.toString() + " reported OPEN on server=" + serverName +
-                    " but state has otherwise AND NO procedure is running");
-                }
-              } catch (UnexpectedStateException e) {
-                LOG.warn(regionNode.toString() + " reported unexpteced OPEN: " + e.getMessage(), e);
-              }
-            }
-          } else if (!regionNode.isInState(State.CLOSING, State.SPLITTING)) {
-            long diff = regionNode.getLastUpdate() - EnvironmentEdgeManager.currentTime();
-            if (diff > 1000/*One Second... make configurable if an issue*/) {
-              // So, we can get report that a region is CLOSED or SPLIT because a heartbeat
-              // came in at about same time as a region transition. Make sure there is some
-              // elapsed time between killing remote server.
-              throw new UnexpectedStateException(regionNode.toString() +
-                " reported an unexpected OPEN; time since last update=" + diff);
-            }
-          }
-        }
-      }
-    } catch (UnexpectedStateException e) {
-      LOG.warn("Killing " + serverName + ": " + e.getMessage());
-      killRegionServer(serverNode);
-    }
-  }
-
-  protected boolean waitServerReportEvent(final ServerName serverName, final Procedure proc) {
-    final ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
-    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;
-
-    @VisibleForTesting
-    public 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() {
-      Map<String, RegionState> m = this.ritsOverThreshold;
-      return m != null ? m.size() : 0;
-    }
-
-    public boolean hasRegionsTwiceOverThreshold() {
-      return totalRITsTwiceThreshold > 0;
-    }
-
-    public boolean hasRegionsOverThreshold() {
-      Map<String, RegionState> m = this.ritsOverThreshold;
-      return m != null && !m.isEmpty();
-    }
-
-    public Collection<RegionState> getRegionOverThreshold() {
-      Map<String, RegionState> m = this.ritsOverThreshold;
-      return m != null? m.values(): Collections.EMPTY_SET;
-    }
-
-    public boolean isRegionOverThreshold(final HRegionInfo regionInfo) {
-      Map<String, RegionState> m = this.ritsOverThreshold;
-      return m != null? m.containsKey(regionInfo.getEncodedName()): false;
-    }
-
-    public boolean isRegionTwiceOverThreshold(final HRegionInfo regionInfo) {
-      Map<String, RegionState> m = this.ritsOverThreshold;
-      if (m == null) return false;
-      final RegionState state = m.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 = regionStates.getRegionNode(regionInfo);
-    //if (regionNode.isStuck()) {
-    LOG.warn("TODO Handle 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 = regionStates.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;
-              regionStates.addRegionToServer(regionLocation, regionNode);
-            } else if (state == State.OFFLINE || regionInfo.isOffline()) {
-              regionStates.addToOfflineRegions(regionNode);
-            } else {
-              // These regions should have a procedure in replay
-              regionStates.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 are put 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: regionStates.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 not running");
-    }
-
-    // TODO: can we avoid throwing an exception if hri is already loaded?
-    //       at the moment we bypass only meta
-    boolean meta = isMetaRegion(hri);
-    boolean cleanup = isFailoverCleanupDone();
-    if (!isMetaRegion(hri) && !isFailoverCleanupDone()) {
-      String msg = "Master not fully online; hbase:meta=" + meta + ", failoverCleanup=" + cleanup;
-      throw new PleaseHoldException(msg);
-    }
-  }
-
-  // ============================================================================================
-  //  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
-    final RegionStateNode node = regionStates.getRegionNode(regionInfo);
-    if (node != null) node.offline();
-  }
-
-  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 regionStates.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 = regionStates.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 regionStates.addRegionInTransition(regionNode, procedure);
-  }
-
-  protected void removeRegionInTransition(final RegionStateNode regionNode,
-      final RegionTransitionProcedure procedure) {
-    regionStates.removeRegionInTransition(regionNode, procedure);
-  }
-
-  public boolean hasRegionsInTransition() {
-    return regionStates.hasRegionsInTransition();
-  }
-
-  public List<RegionStateNode> getRegionsInTransition() {
-    return regionStates.getRegionsInTransition();
-  }
-
-  public List<HRegionInfo> getAssignedRegions() {
-    return regionStates.getAssignedRegions();
-  }
-
-  public HRegionInfo getRegionInfo(final byte[] regionName) {
-    final RegionStateNode regionState = regionStates.getRegionNodeFromName(regionName);
-    return regionState != null ? regionState.getRegionInfo() : null;
-  }
-
-  // ============================================================================================
-  //  TODO: Region Status update
-  // ============================================================================================
-  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 markRegionAsOpening(final RegionStateNode regionNode) throws IOException {
-    synchronized (regionNode) {
-      State state = regionNode.transitionState(State.OPENING, RegionStates.STATES_EXPECTED_ON_OPEN);
-      regionStates.addRegionToServer(regionNode.getRegionLocation(), regionNode);
-      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), state,
-        regionNode.getRegionLocation(), regionNode.getLastHost(), HConstants.NO_SEQNUM,
-        regionNode.getProcedure().getProcId());
-    }
-
-    // update the operation count metrics
-    metrics.incrementOperationCounter();
-  }
-
-  public void undoRegionAsOpening(final RegionStateNode regionNode) {
-    boolean opening = false;
-    synchronized (regionNode) {
-      if (regionNode.isInState(State.OPENING)) {
-        opening = true;
-        regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
-      }
-      // Should we update hbase:meta?
-    }
-    if (opening) {
-      // TODO: Metrics. Do opposite of metrics.incrementOperationCounter();
-    }
-  }
-
-  public void markRegionAsOpened(final RegionStateNode regionNode) throws IOException {
-    final HRegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      State state = regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
-      if (isMetaRegion(hri)) {
-        setMetaInitialized(hri, true);
-      }
-      regionStates.addRegionToServer(regionNode.getRegionLocation(), regionNode);
-      // TODO: OPENING Updates hbase:meta too... we need to do both here and there?
-      // That is a lot of hbase:meta writing.
-      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), state,
-        regionNode.getRegionLocation(), regionNode.getLastHost(), regionNode.getOpenSeqNum(),
-        regionNode.getProcedure().getProcId());
-      sendRegionOpenedNotification(hri, regionNode.getRegionLocation());
-      // update assignment metrics
-      if (regionNode.getProcedure() != null) {
-        metrics.updateAssignTime(regionNode.getProcedure().elapsedTime());
-      }
-    }
-  }
-
-  public void markRegionAsClosing(final RegionStateNode regionNode) throws IOException {
-    final HRegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      State state = regionNode.transitionState(State.CLOSING, RegionStates.STATES_EXPECTED_ON_CLOSE);
-      // Set meta has not initialized early. so people trying to create/edit tables will wait
-      if (isMetaRegion(hri)) {
-        setMetaInitialized(hri, false);
-      }
-      regionStates.addRegionToServer(regionNode.getRegionLocation(), regionNode);
-      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), state,
-        regionNode.getRegionLocation(), regionNode.getLastHost(), HConstants.NO_SEQNUM,
-        regionNode.getProcedure().getProcId());
-    }
-
-    // update the operation count metrics
-    metrics.incrementOperationCounter();
-  }
-
-  public void undoRegionAsClosing(final RegionStateNode regionNode) throws IOException {
-    // TODO: Metrics. Do opposite of metrics.incrementOperationCounter();
-    // There is nothing to undo?
-  }
-
-  public void markRegionAsClosed(final RegionStateNode regionNode) throws IOException {
-    final HRegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      State state = regionNode.transitionState(State.CLOSED, RegionStates.STATES_EXPECTED_ON_CLOSE);
-      regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
-      regionNode.setLastHost(regionNode.getRegionLocation());
-      regionNode.setRegionLocation(null);
-      regionStateStore.updateRegionLocation(regionNode.getRegionInfo(), state,
-        regionNode.getRegionLocation()/*null*/, regionNode.getLastHost(),
-        HConstants.NO_SEQNUM, regionNode.getProcedure().getProcId());
-      sendRegionClosedNotification(hri);
-      // Update assignment metrics
-      if (regionNode.getProcedure() != null) {
-        metrics.updateUnassignTime(regionNode.getProcedure().elapsedTime());
-      }
-    }
-  }
-
-  public void markRegionAsSplit(final HRegionInfo parent, final ServerName serverName,
-      final HRegionInfo daughterA, final HRegionInfo daughterB)
-  throws IOException {
-    // Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
-    // The parent stays in regionStates until cleared when removed by CatalogJanitor.
-    // Update its state in regionStates to it shows as offline and split when read
-    // later figuring what regions are in a table and what are not: see
-    // regionStates#getRegionsOfTable
-    final RegionStateNode node = regionStates.getOrCreateRegionNode(parent);
-    node.setState(State.SPLIT);
-    regionStateStore.splitRegion(parent, daughterA, daughterB, serverName);
-    if (shouldAssignFavoredNodes(parent)) {
-      List<ServerName> onlineServers = this.master.getServerManager().getOnlineServersList();
-      ((FavoredNodesPromoter)getBalancer()).
-          generateFavoredNodesForDaughter(onlineServers, parent, daughterA, daughterB);
-    }
-  }
-
-  /**
-   * When called here, the merge has happened. The two merged regions have been
-   * unassigned and the above markRegionClosed has been called on each so they have been
-   * disassociated from a hosting Server. The merged region will be open after this call. The
-   * merged regions are removed from hbase:meta below> Later they are deleted from the filesystem
-   * by the catalog janitor running against hbase:meta. It notices when the merged region no
-   * longer holds references to the old regions.
-   */
-  public void markRegionAsMerged(final HRegionInfo child, final ServerName serverName,
-      final HRegionInfo mother, final HRegionInfo father) throws IOException {
-    final RegionStateNode node = regionStates.getOrCreateRegionNode(child);
-    node.setState(State.MERGED);
-    regionStates.deleteRegion(mother);
-    regionStates.deleteRegion(father);
-    regionStateStore.mergeRegions(child, mother, father, serverName);
-    if (shouldAssignFavoredNodes(child)) {
-      ((FavoredNodesPromoter)getBalancer()).
-        generateFavoredNodesForMergedRegion(child, mother, father);
-    }
-  }
-
-  /*
-   * 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);
-  }
-
-  // ============================================================================================
-  //  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;
-    }
-
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("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();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("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 = regionStates.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 = regionStates.getServerNode(serverName);
-    return node != null ? node.getVersionNumber() : 0;
-  }
-
-  public void killRegionServer(final ServerName serverName) {
-    final ServerStateNode serverNode = regionStates.getServerNode(serverName);
-    killRegionServer(serverNode);
-  }
-
-  public void killRegionServer(final ServerStateNode serverNode) {
-    /** Don't do this. Messes up accounting. Let ServerCrashProcedure do this.
-    for (RegionStateNode regionNode: serverNode.getRegions()) {
-      regionNode.offline();
-    }*/
-    master.getServerManager().expireServer(serverNode.getServerName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/FailedRemoteDispatchException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/FailedRemoteDispatchException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/FailedRemoteDispatchException.java
deleted file mode 100644
index 111b525..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/FailedRemoteDispatchException.java
+++ /dev/null
@@ -1,33 +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.assignment;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Used internally signaling failed queue of a remote procedure
- * operation.
- */
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-public class FailedRemoteDispatchException extends HBaseIOException {
-  public FailedRemoteDispatchException(String msg) {
-    super(msg);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
deleted file mode 100644
index c7d97ee..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
+++ /dev/null
@@ -1,170 +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.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.MetaTableAccessor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
-import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState;
-
-/**
- * GC regions that have been Merged.
- * Caller determines if it is GC time. This Procedure does not check.
- * <p>This is a Table Procedure. We take a read lock on the Table.
- * We do NOT keep a lock for the life of this procedure. The subprocedures
- * take locks on the Regions they are purging.
- */
-@InterfaceAudience.Private
-public class GCMergedRegionsProcedure
-extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
-  private static final Log LOG = LogFactory.getLog(GCMergedRegionsProcedure.class);
-  private HRegionInfo father;
-  private HRegionInfo mother;
-  private HRegionInfo mergedChild;
-
-  public GCMergedRegionsProcedure(final MasterProcedureEnv env,
-      final HRegionInfo mergedChild,
-      final HRegionInfo father,
-      final HRegionInfo mother) {
-    super(env);
-    this.father = father;
-    this.mother = mother;
-    this.mergedChild = mergedChild;
-  }
-
-  public GCMergedRegionsProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.MERGED_REGIONS_GC;
-  }
-
-  @Override
-  protected Flow executeFromState(MasterProcedureEnv env, GCMergedRegionsState state)
-  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-    try {
-      switch (state) {
-      case GC_MERGED_REGIONS_PREPARE:
-        // Nothing to do to prepare.
-        setNextState(GCMergedRegionsState.GC_MERGED_REGIONS_PURGE);
-        break;
-      case GC_MERGED_REGIONS_PURGE:
-        addChildProcedure(createGCRegionProcedures(env));
-        setNextState(GCMergedRegionsState.GC_REGION_EDIT_METADATA);
-        break;
-      case GC_REGION_EDIT_METADATA:
-        MetaTableAccessor.deleteMergeQualifiers(env.getMasterServices().getConnection(), mergedChild);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (IOException ioe) {
-      // TODO: This is going to spew log?
-      LOG.warn("Error trying to GC merged regions " + this.father.getShortNameToLog() +
-          " & " + this.mother.getShortNameToLog() + "; retrying...", ioe);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) {
-    GCRegionProcedure [] procs = new GCRegionProcedure[2];
-    int index = 0;
-    for (HRegionInfo hri: new HRegionInfo [] {this.father, this.mother}) {
-      GCRegionProcedure proc = new GCRegionProcedure(env, hri);
-      proc.setOwner(env.getRequestUser().getShortName());
-      procs[index++] = proc;
-    }
-    return procs;
-  }
-
-  @Override
-  protected void rollbackState(MasterProcedureEnv env, GCMergedRegionsState state)
-  throws IOException, InterruptedException {
-    // no-op
-  }
-
-  @Override
-  protected GCMergedRegionsState getState(int stateId) {
-    return GCMergedRegionsState.forNumber(stateId);
-  }
-
-  @Override
-  protected int getStateId(GCMergedRegionsState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected GCMergedRegionsState getInitialState() {
-    return GCMergedRegionsState.GC_MERGED_REGIONS_PREPARE;
-  }
-
-  @Override
-  protected void serializeStateData(OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-    final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
-        MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
-        setParentA(HRegionInfo.convert(this.father)).
-        setParentB(HRegionInfo.convert(this.mother)).
-        setMergedChild(HRegionInfo.convert(this.mergedChild));
-    msg.build().writeDelimitedTo(stream);
-  }
-
-  @Override
-  protected void deserializeStateData(InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-    final MasterProcedureProtos.GCMergedRegionsStateData msg =
-        MasterProcedureProtos.GCMergedRegionsStateData.parseDelimitedFrom(stream);
-    this.father = HRegionInfo.convert(msg.getParentA());
-    this.mother = HRegionInfo.convert(msg.getParentB());
-    this.mergedChild = HRegionInfo.convert(msg.getMergedChild());
-  }
-
-  @Override
-  public void toStringClassDetails(StringBuilder sb) {
-    sb.append(getClass().getSimpleName());
-    sb.append(" child=");
-    sb.append(this.mergedChild.getShortNameToLog());
-    sb.append(", father=");
-    sb.append(this.father.getShortNameToLog());
-    sb.append(", mother=");
-    sb.append(this.mother.getShortNameToLog());
-  }
-
-  @Override
-  public TableName getTableName() {
-    return this.mergedChild.getTable();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
deleted file mode 100644
index 3874232..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
+++ /dev/null
@@ -1,155 +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.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.fs.FileSystem;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.MetaTableAccessor;
-import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
-import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
-
-import com.google.common.collect.Lists;
-
-/**
- * GC a Region that is no longer in use. It has been split or merged away.
- * Caller determines if it is GC time. This Procedure does not check.
- * <p>This is a Region StateMachine Procedure. We take a read lock on the Table and then
- * exclusive on the Region.
- */
-@InterfaceAudience.Private
-public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCRegionState> {
-  private static final Log LOG = LogFactory.getLog(GCRegionProcedure.class);
-
-  public GCRegionProcedure(final MasterProcedureEnv env, final HRegionInfo hri) {
-    super(env, hri);
-  }
-
-  public GCRegionProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  @Override
-  public TableOperationType getTableOperationType() {
-    return TableOperationType.REGION_GC;
-  }
-
-  @Override
-  protected Flow executeFromState(MasterProcedureEnv env, GCRegionState state)
-  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
-    MasterServices masterServices = env.getMasterServices();
-    try {
-      switch (state) {
-      case GC_REGION_PREPARE:
-        // Nothing to do to prepare.
-        setNextState(GCRegionState.GC_REGION_ARCHIVE);
-        break;
-      case GC_REGION_ARCHIVE:
-        FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
-        if (HFileArchiver.exists(masterServices.getConfiguration(), fs, getRegion())) {
-          if (LOG.isDebugEnabled()) LOG.debug("Archiving region=" + getRegion().getShortNameToLog());
-          HFileArchiver.archiveRegion(masterServices.getConfiguration(), fs, getRegion());
-        }
-        setNextState(GCRegionState.GC_REGION_PURGE_METADATA);
-        break;
-      case GC_REGION_PURGE_METADATA:
-        // TODO: Purge metadata before removing from HDFS? This ordering is copied
-        // from CatalogJanitor.
-        AssignmentManager am = masterServices.getAssignmentManager();
-        if (am != null) {
-          if (am.getRegionStates() != null) {
-            am.getRegionStates().deleteRegion(getRegion());
-          }
-        }
-        MetaTableAccessor.deleteRegion(masterServices.getConnection(), getRegion());
-        masterServices.getServerManager().removeRegion(getRegion());
-        FavoredNodesManager fnm = masterServices.getFavoredNodesManager();
-        if (fnm != null) {
-          fnm.deleteFavoredNodesForRegions(Lists.newArrayList(getRegion()));
-        }
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
-      }
-    } catch (IOException ioe) {
-      // TODO: This is going to spew log?
-      LOG.warn("Error trying to GC " + getRegion().getShortNameToLog() + "; retrying...", ioe);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  @Override
-  protected void rollbackState(MasterProcedureEnv env, GCRegionState state) throws IOException, InterruptedException {
-    // no-op
-  }
-
-  @Override
-  protected GCRegionState getState(int stateId) {
-    return GCRegionState.forNumber(stateId);
-  }
-
-  @Override
-  protected int getStateId(GCRegionState state) {
-    return state.getNumber();
-  }
-
-  @Override
-  protected GCRegionState getInitialState() {
-    return GCRegionState.GC_REGION_PREPARE;
-  }
-
-  @Override
-  protected void serializeStateData(OutputStream stream) throws IOException {
-    super.serializeStateData(stream);
-    // Double serialization of regionname. Superclass is also serializing. Fix.
-    final MasterProcedureProtos.GCRegionStateData.Builder msg =
-        MasterProcedureProtos.GCRegionStateData.newBuilder()
-        .setRegionInfo(HRegionInfo.convert(getRegion()));
-    msg.build().writeDelimitedTo(stream);
-  }
-
-  @Override
-  protected void deserializeStateData(InputStream stream) throws IOException {
-    super.deserializeStateData(stream);
-    final MasterProcedureProtos.GCRegionStateData msg =
-        MasterProcedureProtos.GCRegionStateData.parseDelimitedFrom(stream);
-    setRegion(HRegionInfo.convert(msg.getRegionInfo()));
-  }
-
-  @Override
-  protected org.apache.hadoop.hbase.procedure2.Procedure.LockState acquireLock(MasterProcedureEnv env) {
-    return super.acquireLock(env);
-  }
-}
\ No newline at end of file


[17/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..929cd4e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java
@@ -0,0 +1,122 @@
+/**
+ *
+ * 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/a3c5a744/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
new file mode 100644
index 0000000..d8c511e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/BulkReOpen.java
@@ -0,0 +1,136 @@
+/**
+ *
+ * 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/a3c5a744/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 4775a0a..affd44c 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
@@ -27,6 +27,7 @@ import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
@@ -38,15 +39,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.TableName;
+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.master.assignment.GCMergedRegionsProcedure;
-import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -55,8 +52,6 @@ import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Triple;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * A janitor for the catalog tables.  Scans the <code>hbase:meta</code> catalog
  * table on a period looking for unused regions to garbage collect.
@@ -69,7 +64,6 @@ public class CatalogJanitor extends ScheduledChore {
   private final AtomicBoolean enabled = new AtomicBoolean(true);
   private final MasterServices services;
   private final Connection connection;
-  // PID of the last Procedure launched herein. Keep around for Tests.
 
   CatalogJanitor(final MasterServices services) {
     super("CatalogJanitor-" + services.getServerName().toShortString(), services,
@@ -118,13 +112,10 @@ public class CatalogJanitor extends ScheduledChore {
           && !this.services.isInMaintenanceMode()
           && am != null
           && am.isFailoverCleanupDone()
-          && !am.hasRegionsInTransition()) {
+          && am.getRegionStates().getRegionsInTransition().isEmpty()) {
         scan();
       } else {
-        LOG.warn("CatalogJanitor is disabled! Enabled=" + this.enabled.get() +
-            ", maintenanceMode=" + this.services.isInMaintenanceMode() +
-            ", am=" + am + ", failoverCleanupDone=" + (am != null && am.isFailoverCleanupDone()) +
-            ", hasRIT=" + (am != null && am.hasRegionsInTransition()));
+        LOG.warn("CatalogJanitor disabled! Not running scan.");
       }
     } catch (IOException e) {
       LOG.warn("Failed scan of catalog table", e);
@@ -176,7 +167,6 @@ public class CatalogJanitor extends ScheduledChore {
           // Another table, stop scanning
           return false;
         }
-        if (LOG.isTraceEnabled()) LOG.trace("" + 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);
@@ -197,6 +187,8 @@ public class CatalogJanitor extends ScheduledChore {
    * If merged region no longer holds reference to the merge regions, archive
    * merge region on hdfs and perform deleting references in hbase:meta
    * @param mergedRegion
+   * @param regionA
+   * @param regionB
    * @return true if we delete references in merged region on hbase:meta and archive
    *         the files on the file system
    * @throws IOException
@@ -215,12 +207,18 @@ public class CatalogJanitor extends ScheduledChore {
       LOG.warn("Merged region does not exist: " + mergedRegion.getEncodedName());
     }
     if (regionFs == null || !regionFs.hasReferences(htd)) {
-      LOG.debug("Deleting region " + regionA.getShortNameToLog() + " and "
-          + regionB.getShortNameToLog()
+      LOG.debug("Deleting region " + regionA.getRegionNameAsString() + " and "
+          + regionB.getRegionNameAsString()
           + " from fs because merged region no longer holds references");
-      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
-      pe.submitProcedure(new GCMergedRegionsProcedure(pe.getEnvironment(),
-          mergedRegion, regionA, regionB));
+      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionA);
+      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionB);
+      MetaTableAccessor.deleteMergeQualifiers(services.getConnection(), mergedRegion);
+      services.getServerManager().removeRegion(regionA);
+      services.getServerManager().removeRegion(regionB);
+      FavoredNodesManager fnm = this.services.getFavoredNodesManager();
+      if (fnm != null) {
+        fnm.deleteFavoredNodesForRegions(Lists.newArrayList(regionA, regionB));
+      }
       return true;
     }
     return false;
@@ -229,21 +227,22 @@ public class CatalogJanitor extends ScheduledChore {
   /**
    * Run janitorial scan of catalog <code>hbase:meta</code> table looking for
    * garbage to collect.
-   * @return number of archiving jobs started.
+   * @return number of cleaned regions
    * @throws IOException
    */
   int scan() throws IOException {
-    int result = 0;
     try {
       if (!alreadyRunning.compareAndSet(false, true)) {
         LOG.debug("CatalogJanitor already running");
-        return result;
+        return 0;
       }
       Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> scanTriple =
         getMergedRegionsAndSplitParents();
+      int count = scanTriple.getFirst();
       /**
        * clean merge regions first
        */
+      int mergeCleaned = 0;
       Map<HRegionInfo, Result> mergedRegions = scanTriple.getSecond();
       for (Map.Entry<HRegionInfo, Result> e : mergedRegions.entrySet()) {
         if (this.services.isInMaintenanceMode()) {
@@ -256,13 +255,13 @@ public class CatalogJanitor extends ScheduledChore {
         HRegionInfo regionB = p.getSecond();
         if (regionA == null || regionB == null) {
           LOG.warn("Unexpected references regionA="
-              + (regionA == null ? "null" : regionA.getShortNameToLog())
+              + (regionA == null ? "null" : regionA.getRegionNameAsString())
               + ",regionB="
-              + (regionB == null ? "null" : regionB.getShortNameToLog())
-              + " in merged region " + e.getKey().getShortNameToLog());
+              + (regionB == null ? "null" : regionB.getRegionNameAsString())
+              + " in merged region " + e.getKey().getRegionNameAsString());
         } else {
           if (cleanMergeRegion(e.getKey(), regionA, regionB)) {
-            result++;
+            mergeCleaned++;
           }
         }
       }
@@ -272,6 +271,7 @@ public class CatalogJanitor extends ScheduledChore {
       Map<HRegionInfo, Result> splitParents = scanTriple.getThird();
 
       // Now work on our list of found parents. See if any we can clean up.
+      int splitCleaned = 0;
       // regions whose parents are still around
       HashSet<String> parentNotCleaned = new HashSet<>();
       for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
@@ -281,8 +281,8 @@ public class CatalogJanitor extends ScheduledChore {
         }
 
         if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
-              cleanParent(e.getKey(), e.getValue())) {
-            result++;
+            cleanParent(e.getKey(), e.getValue())) {
+          splitCleaned++;
         } else {
           // We could not clean the parent, so it's daughters should not be
           // cleaned either (HBASE-6160)
@@ -292,7 +292,16 @@ public class CatalogJanitor extends ScheduledChore {
           parentNotCleaned.add(daughters.getSecond().getEncodedName());
         }
       }
-      return result;
+      if ((mergeCleaned + splitCleaned) != 0) {
+        LOG.info("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
+            + " unreferenced merged region(s) and " + splitCleaned
+            + " unreferenced parent region(s)");
+      } else if (LOG.isTraceEnabled()) {
+        LOG.trace("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
+            + " unreferenced merged region(s) and " + splitCleaned
+            + " unreferenced parent region(s)");
+      }
+      return mergeCleaned + splitCleaned;
     } finally {
       alreadyRunning.set(false);
     }
@@ -334,30 +343,34 @@ public class CatalogJanitor extends ScheduledChore {
    */
   boolean cleanParent(final HRegionInfo parent, Result rowContent)
   throws IOException {
+    boolean result = false;
     // Check whether it is a merged region and not clean reference
     // No necessary to check MERGEB_QUALIFIER because these two qualifiers will
     // be inserted/deleted together
-    if (rowContent.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
+    if (rowContent.getValue(HConstants.CATALOG_FAMILY,
+        HConstants.MERGEA_QUALIFIER) != null) {
       // wait cleaning merge region first
-      return false;
+      return result;
     }
     // Run checks on each daughter split.
     PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
     Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
     Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
     if (hasNoReferences(a) && hasNoReferences(b)) {
-      String daughterA = daughters.getFirst() != null?
-          daughters.getFirst().getShortNameToLog(): "null";
-      String daughterB = daughters.getSecond() != null?
-          daughters.getSecond().getShortNameToLog(): "null";
-      LOG.debug("Deleting region " + parent.getShortNameToLog() +
-        " because daughters -- " + daughterA + ", " + daughterB +
-        " -- no longer hold references");
-      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
-      pe.submitProcedure(new GCRegionProcedure(pe.getEnvironment(), parent));
-      return true;
+      LOG.debug("Deleting region " + parent.getRegionNameAsString() +
+        " because daughter splits no longer hold references");
+      FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
+      if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent);
+      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
+      MetaTableAccessor.deleteRegion(this.connection, parent);
+      services.getServerManager().removeRegion(parent);
+      FavoredNodesManager fnm = this.services.getFavoredNodesManager();
+      if (fnm != null) {
+        fnm.deleteFavoredNodesForRegions(Lists.newArrayList(parent));
+      }
+      result = true;
     }
-    return false;
+    return result;
   }
 
   /**
@@ -456,4 +469,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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
index 34a7633..faceba2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/DeadServer.java
@@ -61,7 +61,7 @@ public class DeadServer {
   /**
    * Whether a dead server is being processed currently.
    */
-  private volatile boolean processing = false;
+  private boolean processing = false;
 
   /**
    * A dead server that comes back alive has a different start code. The new start code should be
@@ -123,14 +123,14 @@ public class DeadServer {
    * @param sn ServerName for the dead server.
    */
   public synchronized void notifyServer(ServerName sn) {
-    if (LOG.isTraceEnabled()) { LOG.trace("Started processing " + sn); }
+    if (LOG.isDebugEnabled()) { LOG.debug("Started processing " + sn); }
     processing = true;
     numProcessing++;
   }
 
   public synchronized void finish(ServerName sn) {
     numProcessing--;
-    if (LOG.isTraceEnabled()) LOG.trace("Finished " + sn + "; numProcessing=" + numProcessing);
+    if (LOG.isDebugEnabled()) LOG.debug("Finished " + sn + "; numProcessing=" + numProcessing);
 
     assert numProcessing >= 0: "Number of dead servers in processing should always be non-negative";
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..fc3607f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/GeneralBulkAssigner.java
@@ -0,0 +1,213 @@
+/**
+ * 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/a3c5a744/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 83f5a1c..4dd6353 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
@@ -36,8 +36,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -68,6 +66,7 @@ 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.ScheduledChore;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -91,10 +90,6 @@ 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;
@@ -115,15 +110,16 @@ 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.MasterProcedureScheduler;
 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;
@@ -346,6 +342,7 @@ 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;
@@ -446,6 +443,19 @@ 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>
@@ -648,6 +658,20 @@ 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;
   }
@@ -670,9 +694,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
     this.splitOrMergeTracker.start();
 
-    // Create Assignment Manager
-    this.assignmentManager = new AssignmentManager(this);
-    this.assignmentManager.start();
+    this.assignmentManager = new AssignmentManager(this, serverManager,
+      this.balancer, this.service, this.metricsMaster, tableStateManager);
 
     this.replicationManager = new ReplicationManager(conf, zooKeeper, this);
 
@@ -863,6 +886,10 @@ 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();
 
@@ -875,8 +902,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     status.markComplete("Initialization successful");
-    LOG.info(String.format("Master has completed initialization %.3fsec",
-       (System.currentTimeMillis() - masterActiveTime) / 1000.0f));
+    LOG.info("Master has completed initialization");
     configurationManager.registerObserver(this.balancer);
     configurationManager.registerObserver(this.hfileCleaner);
 
@@ -985,8 +1011,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);
       }
     }
@@ -1119,6 +1145,12 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
+  protected void sendShutdownInterrupt() {
+    super.sendShutdownInterrupt();
+    stopProcedureExecutor();
+  }
+
+  @Override
   protected void stopServiceThreads() {
     if (masterJettyServer != null) {
       LOG.info("Stopping master jetty server");
@@ -1140,20 +1172,15 @@ 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.");
@@ -1163,9 +1190,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
     final Path walDir = new Path(FSUtils.getWALRootDir(this.conf),
         MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
-    // TODO: No cleaner currently!
-    final Path walArchiveDir = new Path(HFileArchiveUtil.getArchivePath(this.conf),
-        MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
 
     final FileSystem walFs = walDir.getFileSystem(conf);
 
@@ -1179,7 +1203,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     FSUtils.setStoragePolicy(walFs, conf, walDir, HConstants.WAL_STORAGE_POLICY,
       HConstants.DEFAULT_WAL_STORAGE_POLICY);
 
-    procedureStore = new WALProcedureStore(conf, walDir.getFileSystem(conf), walDir, walArchiveDir,
+    procedureStore = new WALProcedureStore(conf, walFs, walDir,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
     procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
     MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
@@ -1194,20 +1218,16 @@ 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;
     }
   }
 
@@ -1237,6 +1257,9 @@ public class HMaster extends HRegionServer implements MasterServices {
       this.mobCompactThread.close();
     }
 
+    if (this.periodicDoMetricsChore != null) {
+      periodicDoMetricsChore.cancel();
+    }
     if (this.quotaObserverChore != null) {
       quotaObserverChore.cancel();
     }
@@ -1297,7 +1320,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().hasRegionsInTransition()) {
+        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount() != 0) {
       try {
         Thread.sleep(100);
       } catch (InterruptedException ie) {
@@ -1308,7 +1331,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     // Throttling by max number regions in transition
     while (!interrupted
         && maxRegionsInTransition > 0
-        && this.assignmentManager.getRegionStates().getRegionsInTransition().size()
+        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
         >= maxRegionsInTransition && System.currentTimeMillis() <= cutoffTime) {
       try {
         // sleep if the number of regions in transition exceeds the limit
@@ -1341,26 +1364,21 @@ 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.hasRegionsInTransition()) {
-        List<RegionStateNode> regionsInTransition = assignmentManager.getRegionsInTransition();
+      // Only allow one balance run at at time.
+      if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
+        Set<RegionState> regionsInTransition =
+          this.assignmentManager.getRegionStates().getRegionsInTransition();
         // if hbase:meta region is in transition, result of assignment cannot be recorded
         // ignore the force flag in that case
-        boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
+        boolean metaInTransition = assignmentManager.getRegionStates().isMetaRegionInTransition();
         String prefix = force && !metaInTransition ? "R" : "Not r";
-        List<RegionStateNode> toPrint = regionsInTransition;
-        int max = 5;
-        boolean truncated = false;
-        if (regionsInTransition.size() > max) {
-          toPrint = regionsInTransition.subList(0, max);
-          truncated = true;
-        }
-        LOG.info(prefix + "unning balancer because " + regionsInTransition.size() +
-          " region(s) in transition: " + toPrint + (truncated? "(truncated list)": ""));
+        LOG.debug(prefix + "unning balancer because " + regionsInTransition.size() +
+          " region(s) in transition: " + org.apache.commons.lang.StringUtils.
+            abbreviate(regionsInTransition.toString(), 256));
         if (!force || metaInTransition) return false;
       }
       if (this.serverManager.areDeadServersInProgress()) {
-        LOG.info("Not running balancer because processing dead regionserver(s): " +
+        LOG.debug("Not running balancer because processing dead regionserver(s): " +
           this.serverManager.getDeadServers());
         return false;
       }
@@ -1385,7 +1403,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());
+              this.assignmentManager.getRegionStates().getAssignmentsByTable(true));
 
       for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
         List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
@@ -1404,7 +1422,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         for (RegionPlan plan: plans) {
           LOG.info("balance " + plan);
           //TODO: bulk assign
-          this.assignmentManager.moveAsync(plan);
+          this.assignmentManager.balance(plan);
           rpCount++;
 
           balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
@@ -1520,59 +1538,6 @@ 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,
@@ -1615,38 +1580,40 @@ public class HMaster extends HRegionServer implements MasterServices {
 
       @Override
       protected String getDescription() {
-        return "MergeTableProcedure";
+        return "DisableTableProcedure";
       }
     });
   }
 
   @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 " + regionInfo.getRegionNameAsString());
+
+        LOG.info(getClientIdAuditPrefix() + " Split region " + regionInfo);
 
         // Execute the operation asynchronously
-        submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
+        submitProcedure(new SplitTableRegionProcedure(procedureExecutor.getEnvironment(),
+            regionInfo, splitRow));
       }
 
       @Override
       protected String getDescription() {
-        return "SplitTableProcedure";
+        return "DisableTableProcedure";
       }
     });
   }
 
-  // Public so can be accessed by tests. Blocks until move is done.
-  // Replace with an async implementation from which you can get
-  // a success/failure result.
-  @VisibleForTesting
+  @VisibleForTesting // Public so can be accessed by tests.
   public void move(final byte[] encodedRegionName,
       final byte[] destServerName) throws HBaseIOException {
     RegionState regionState = assignmentManager.getRegionStates().
@@ -1697,8 +1664,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     // Now we can do the move
     RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
-    assert rp.getDestination() != null: rp.toString() + " " + dest;
-    assert rp.getSource() != null: rp.toString();
 
     try {
       checkInitialized();
@@ -1707,20 +1672,13 @@ public class HMaster extends HRegionServer implements MasterServices {
           return;
         }
       }
-      // Warmup the region on the destination before initiating the move. this call
+      // warmup the region on the destination before initiating the move. this call
       // is synchronous and takes some time. doing it before the source region gets
       // closed
       serverManager.sendRegionWarmup(rp.getDestination(), hri);
 
       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
-      Future<byte []> future = this.assignmentManager.moveAsync(rp);
-      try {
-        // Is this going to work? Will we throw exception on error?
-        // TODO: CompletableFuture rather than this stunted Future.
-        future.get();
-      } catch (InterruptedException | ExecutionException e) {
-        throw new HBaseIOException(e);
-      }
+      this.assignmentManager.balance(rp);
       if (this.cpHost != null) {
         this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
       }
@@ -2059,7 +2017,7 @@ public class HMaster extends HRegionServer implements MasterServices {
           status.cleanup();
         }
       }
-    }, getServerName().toShortString() + ".masterManager"));
+    }, getServerName().toShortString() + ".activeMasterManager"));
   }
 
   private void checkCompression(final HTableDescriptor htd)
@@ -2512,9 +2470,8 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     String clusterId = fileSystemManager != null ?
       fileSystemManager.getClusterId().toString() : null;
-    List<RegionState> regionsInTransition = assignmentManager != null ?
-      assignmentManager.getRegionStates().getRegionsStateInTransition() : null;
-
+    Set<RegionState> regionsInTransition = assignmentManager != null ?
+      assignmentManager.getRegionStates().getRegionsInTransition() : null;
     String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
     boolean balancerOn = loadBalancerTracker != null ?
       loadBalancerTracker.isBalancerOn() : false;
@@ -2722,7 +2679,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
   }
 
-  @Override
   public ProcedureEvent getInitializedEvent() {
     return initialized;
   }
@@ -2833,7 +2789,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @see org.apache.hadoop.hbase.master.HMasterCommandLine
    */
   public static void main(String [] args) {
-    LOG.info("STARTING service '" + HMaster.class.getSimpleName());
+    LOG.info("***** STARTING service '" + HMaster.class.getSimpleName() + "' *****");
     VersionInfo.logVersion();
     new HMasterCommandLine(HMaster.class).doMain(args);
   }
@@ -3278,7 +3234,6 @@ 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/a3c5a744/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 4611982..129fa7a 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
@@ -45,7 +45,7 @@ import edu.umd.cs.findbugs.annotations.Nullable;
  * locations for all Regions in a cluster.
  *
  * <p>This class produces plans for the
- * {@link org.apache.hadoop.hbase.master.assignment.AssignmentManager}
+ * {@link org.apache.hadoop.hbase.master.AssignmentManager}
  * to execute.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 4d18ac9..6064f9b 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
@@ -810,28 +810,6 @@ 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/a3c5a744/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 a48444c..a921ab5 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,6 +24,7 @@ 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;
@@ -32,8 +33,6 @@ 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;
@@ -118,8 +117,9 @@ public class MasterDumpServlet extends StateDumpServlet {
       return;
     }
 
-    for (RegionStateNode rs : am.getRegionsInTransition()) {
-      String rid = rs.getRegionInfo().getEncodedName();
+    Set<RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
+    for (RegionState rs : regionsInTransition) {
+      String rid = rs.getRegion().getRegionNameAsString();
       out.println("Region " + rid + ": " + rs.toDescriptiveString());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 049e659..1988e2d 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -32,8 +33,8 @@ 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;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -107,7 +108,14 @@ public class MasterMetaBootstrap {
   }
 
   private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
-    master.getMasterWalManager().splitMetaLog(currentMetaServer);
+    if (RecoveryMode.LOG_REPLAY == master.getMasterWalManager().getLogRecoveryMode()) {
+      // In log replay mode, we mark hbase:meta region as recovering in ZK
+      master.getMasterWalManager().prepareLogReplay(currentMetaServer,
+        Collections.<HRegionInfo>singleton(HRegionInfo.FIRST_META_REGIONINFO));
+    } else {
+      // In recovered.edits mode: create recovered edits file for hbase:meta server
+      master.getMasterWalManager().splitMetaLog(currentMetaServer);
+    }
   }
 
   private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
@@ -143,9 +151,7 @@ public class MasterMetaBootstrap {
 
     // Work on meta region
     int assigned = 0;
-    // TODO: Unimplemented
-    // long timeout =
-    //   master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
+    long timeout = master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
       status.setStatus("Assigning hbase:meta region");
     } else {
@@ -154,10 +160,37 @@ public class MasterMetaBootstrap {
 
     // Get current meta state from zk.
     RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), replicaId);
-    LOG.debug("meta state from zookeeper: " + metaState);
-    HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-      HRegionInfo.FIRST_META_REGIONINFO, replicaId);
-    assignmentManager.assignMeta(hri, metaState.getServerName());
+    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++;
+    }
 
     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
       // TODO: should we prevent from using state manager before meta was initialized?
@@ -166,6 +199,14 @@ public class MasterMetaBootstrap {
         .setTableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
     }
 
+    if ((RecoveryMode.LOG_REPLAY == master.getMasterWalManager().getLogRecoveryMode())
+        && (!previouslyFailedMetaRSs.isEmpty())) {
+      // replay WAL edits mode need new hbase:meta RS is assigned firstly
+      status.setStatus("replaying log for Meta Region");
+      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
@@ -173,7 +214,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 + ", location="
+    LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", location="
       + master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper(), replicaId));
     status.setStatus("META assigned.");
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 c43a4d1..296d4d7 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
@@ -37,6 +37,7 @@ 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;
@@ -45,7 +46,6 @@ 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;
@@ -54,7 +54,6 @@ 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;
@@ -86,6 +85,7 @@ 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.LockServiceProtos.LockHeartbeatRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockHeartbeatResponse;
@@ -136,6 +136,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 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;
@@ -304,11 +306,7 @@ public class MasterRpcServices extends RSRpcServices
       ClusterStatusProtos.ServerLoad sl = request.getLoad();
       ServerName serverName = ProtobufUtil.toServerName(request.getServer());
       ServerLoad oldLoad = master.getServerManager().getLoad(serverName);
-      ServerLoad newLoad = new ServerLoad(sl);
-      master.getServerManager().regionServerReport(serverName, newLoad);
-      int version = VersionInfoUtil.getCurrentClientVersionNumber();
-      master.getAssignmentManager().reportOnlineRegions(serverName,
-        version, newLoad.getRegionsLoad().keySet());
+      master.getServerManager().regionServerReport(serverName, new ServerLoad(sl));
       if (sl != null && master.metricsMaster != null) {
         // Up our metrics.
         master.metricsMaster.incrementRequests(sl.getTotalNumberOfRequests()
@@ -381,25 +379,25 @@ public class MasterRpcServices extends RSRpcServices
   public AssignRegionResponse assignRegion(RpcController controller,
       AssignRegionRequest req) throws ServiceException {
     try {
-      master.checkInitialized();
+      final byte [] regionName = req.getRegion().getValue().toByteArray();
+      RegionSpecifierType type = req.getRegion().getType();
+      AssignRegionResponse arr = AssignRegionResponse.newBuilder().build();
 
-      final RegionSpecifierType type = req.getRegion().getType();
+      master.checkInitialized();
       if (type != RegionSpecifierType.REGION_NAME) {
         LOG.warn("assignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
           + " actual: " + type);
       }
-
-      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();
+      RegionStates regionStates = master.getAssignmentManager().getRegionStates();
+      HRegionInfo regionInfo = regionStates.getRegionInfo(regionName);
+      if (regionInfo == null) throw new UnknownRegionException(Bytes.toString(regionName));
       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);
@@ -410,7 +408,6 @@ public class MasterRpcServices extends RSRpcServices
     }
   }
 
-
   @Override
   public BalanceResponse balance(RpcController controller,
       BalanceRequest request) throws ServiceException {
@@ -630,7 +627,8 @@ 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(
@@ -1217,24 +1215,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();
-
-      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));
-
+      Pair<HRegionInfo, ServerName> pair =
+        MetaTableAccessor.getRegion(master.getConnection(), regionName);
+      if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
+      HRegionInfo hri = pair.getFirst();
       if (master.cpHost != null) {
         master.cpHost.preRegionOffline(hri);
       }
       LOG.info(master.getClientIdAuditPrefix() + " offline " + hri.getRegionNameAsString());
-      master.getAssignmentManager().offlineRegion(hri);
+      master.getAssignmentManager().regionOffline(hri);
       if (master.cpHost != null) {
         master.cpHost.postRegionOffline(hri);
       }
@@ -1419,7 +1417,26 @@ public class MasterRpcServices extends RSRpcServices
       ReportRegionStateTransitionRequest req) throws ServiceException {
     try {
       master.checkServiceStarted();
-      return master.getAssignmentManager().reportRegionStateTransition(req);
+      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();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
@@ -2008,34 +2025,4 @@ 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);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 781e907..4924d72 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,9 +32,7 @@ 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;
@@ -42,14 +40,11 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockInfo;
-import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 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 com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.Service;
 
 /**
@@ -128,12 +123,6 @@ public interface MasterServices extends Server {
   ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor();
 
   /**
-   * @return Tripped when Master has finished initialization.
-   */
-  @VisibleForTesting
-  public ProcedureEvent getInitializedEvent();
-
-  /**
    * Check table is modifiable; i.e. exists and is offline.
    * @param tableName Name of table to check.
    * @throws TableNotDisabledException
@@ -277,23 +266,6 @@ 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
@@ -429,8 +401,6 @@ 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/a3c5a744/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 928702e..105fa29 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
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -39,13 +41,12 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * This class abstracts a bunch of operations the HMaster needs
  * when splitting log files e.g. finding log files, dirs etc.
@@ -331,4 +332,16 @@ public class MasterWalManager {
       }
     }
   }
+
+  /**
+   * The function is used in SSH to set recovery mode based on configuration after all outstanding
+   * log split tasks drained.
+   */
+  public void setLogRecoveryMode() throws IOException {
+    this.splitLogManager.setRecoveryMode(false);
+  }
+
+  public RecoveryMode getLogRecoveryMode() {
+    return this.splitLogManager.getRecoveryMode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 c7ce9a9..40e79ae 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,6 +21,7 @@ package org.apache.hadoop.hbase.master;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
 
 public class MetricsAssignmentManager {
+
   private final MetricsAssignmentManagerSource assignmentManagerSource;
 
   public MetricsAssignmentManager() {
@@ -32,11 +33,19 @@ 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(final int ritCount) {
+  public void updateRITCount(int ritCount) {
     assignmentManagerSource.setRIT(ritCount);
   }
 
@@ -45,15 +54,14 @@ public class MetricsAssignmentManager {
    * as defined by the property rit.metrics.threshold.time.
    * @param ritCountOverThreshold
    */
-  public void updateRITCountOverThreshold(final int ritCountOverThreshold) {
+  public void updateRITCountOverThreshold(int ritCountOverThreshold) {
     assignmentManagerSource.setRITCountOverThreshold(ritCountOverThreshold);
   }
-
   /**
    * update the timestamp for oldest region in transition metrics.
    * @param timestamp
    */
-  public void updateRITOldestAge(final long timestamp) {
+  public void updateRITOldestAge(long timestamp) {
     assignmentManagerSource.setRITOldestAge(timestamp);
   }
 
@@ -64,27 +72,4 @@ 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);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NoSuchProcedureException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NoSuchProcedureException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NoSuchProcedureException.java
deleted file mode 100644
index e119e88..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/NoSuchProcedureException.java
+++ /dev/null
@@ -1,33 +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.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-// Based on HBaseIOE rather than PE because easier to integrate when an IOE.
-public class NoSuchProcedureException extends HBaseIOException {
-  public NoSuchProcedureException() {
-    super();
-  }
-
-  public NoSuchProcedureException(String s) {
-    super(s);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
index 17eb346..cd6b313 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlan.java
@@ -135,8 +135,8 @@ public class RegionPlan implements Comparable<RegionPlan> {
 
   @Override
   public String toString() {
-    return "hri=" + this.hri.getRegionNameAsString() + ", source=" +
+    return "hri=" + this.hri.getRegionNameAsString() + ", src=" +
       (this.source == null? "": this.source.toString()) +
-      ", destination=" + (this.dest == null? "": this.dest.toString());
+      ", dest=" + (this.dest == null? "": this.dest.toString());
   }
 }


[10/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 71c6b89..2703947 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,40 +19,55 @@ 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.Iterator;
+import java.util.HashSet;
 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.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 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.MasterServices;
 import org.apache.hadoop.hbase.master.MasterWalManager;
-import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionTransitionProcedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo;
+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
  * ServerShutdownHandler.
  *
- * <p>The procedure flow varies dependent on whether meta is assigned and if we are to split logs.
+ * <p>The procedure flow varies dependent on whether meta is assigned, if we are
+ * doing distributed log replay versus distributed log splitting, and if we are to split logs at
+ * all.
+ *
+ * <p>This procedure asks that all crashed servers get processed equally; we yield after the
+ * 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>We come in here after ServerManager has noticed a server has expired. Procedures
- * queued on the rpc should have been notified about fail and should be concurrently
- * getting themselves ready to assign elsewhere.
+ * <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>
@@ -60,6 +75,36 @@ 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;
@@ -72,8 +117,14 @@ implements ServerProcedureInterface {
   /**
    * Regions that were on the crashed server.
    */
-  private List<HRegionInfo> regionsOnCrashedServer;
+  private Set<HRegionInfo> regionsOnCrashedServer;
 
+  /**
+   * Regions assigned. Usually some subset of {@link #regionsOnCrashedServer}.
+   */
+  private List<HRegionInfo> regionsAssigned;
+
+  private boolean distributedLogReplay = false;
   private boolean carryingMeta = false;
   private boolean shouldSplitWal;
 
@@ -113,11 +164,20 @@ implements ServerProcedureInterface {
     super();
   }
 
+  private void throwProcedureYieldException(final String msg) throws ProcedureYieldException {
+    String logMsg = msg + "; cycle=" + this.cycles + ", running for " +
+        StringUtils.formatTimeDiff(System.currentTimeMillis(), getSubmittedTime());
+    // The procedure executor logs ProcedureYieldException at trace level. For now, log these
+    // yields for server crash processing at DEBUG. Revisit when stable.
+    if (LOG.isDebugEnabled()) LOG.debug(logMsg);
+    throw new ProcedureYieldException(logMsg);
+  }
+
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, ServerCrashState state)
-      throws ProcedureSuspendedException, ProcedureYieldException {
+      throws ProcedureYieldException {
     if (LOG.isTraceEnabled()) {
-      LOG.trace(state  + " " + this + "; cycles=" + this.cycles);
+      LOG.trace(state);
     }
     // Keep running count of cycles
     if (state.ordinal() != this.previousState) {
@@ -126,7 +186,11 @@ implements ServerProcedureInterface {
     } else {
       this.cycles++;
     }
-    final MasterServices services = env.getMasterServices();
+    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");
+    }
     // HBASE-14802
     // If we have not yet notified that we are processing a dead server, we should do now.
     if (!notifiedDeadServer) {
@@ -137,61 +201,102 @@ implements ServerProcedureInterface {
     try {
       switch (state) {
       case SERVER_CRASH_START:
-        LOG.info("Start " + this);
+        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 (env.getAssignmentManager().waitMetaInitialized(this)) {
-          throw new ProcedureSuspendedException();
+        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");
         }
-
-        this.regionsOnCrashedServer = services.getAssignmentManager().getRegionStates()
-          .getServerRegionInfoSet(serverName);
-        // Where to go next? Depends on whether we should split logs at all or
-        // if we should do distributed log splitting.
+        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).
         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:
-        processMeta(env);
+        // If we fail processing hbase:meta, yield.
+        if (!processMeta(env)) {
+          throwProcedureYieldException("Waiting on regions-in-transition to clear");
+        }
         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);
-        setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
+        // 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);
         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.
-        // Filter out meta regions. Those are handled elsewhere in this procedure.
-        // Filter changes this.regionsOnCrashedServer.
-        if (filterDefaultMetaRegions(regionsOnCrashedServer)) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Assigning regions " +
-              HRegionInfo.getShortNameToLog(regionsOnCrashedServer) + ", " + this +
-              "; cycles=" + this.cycles);
+        boolean regions = regionsToAssign != null && !regionsToAssign.isEmpty();
+        if (regions) {
+          this.regionsAssigned = regionsToAssign;
+          if (!assign(env, regionsToAssign)) {
+            throwProcedureYieldException("Failed assign; will retry");
           }
-          handleRIT(env, regionsOnCrashedServer);
-          addChildProcedure(env.getAssignmentManager().
-              createAssignProcedures(regionsOnCrashedServer, true));
         }
-        setNextState(ServerCrashState.SERVER_CRASH_FINISH);
+        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");
+          }
+        }
+        setNextState(ServerCrashState.SERVER_CRASH_SPLIT_LOGS);
         break;
 
       case SERVER_CRASH_FINISH:
+        LOG.info("Finished processing of crashed " + serverName);
         services.getServerManager().getDeadServers().finish(serverName);
         return Flow.NO_MORE_STATE;
 
@@ -199,7 +304,11 @@ implements ServerProcedureInterface {
         throw new UnsupportedOperationException("unhandled state=" + state);
       }
     } catch (IOException e) {
-      LOG.warn("Failed state=" + state + ", retry " + this + "; cycles=" + this.cycles, 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;
   }
@@ -209,60 +318,96 @@ implements ServerProcedureInterface {
    * @param env
    * @throws IOException
    */
-  private void start(final MasterProcedureEnv env) throws IOException {}
+  private void start(final MasterProcedureEnv env) throws IOException {
+    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;
+  }
 
   /**
    * @param env
+   * @return False if we fail to assign and split logs on meta ('process').
    * @throws IOException
    * @throws InterruptedException
    */
-  private void processMeta(final MasterProcedureEnv env) throws IOException {
+  private boolean 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) {
-      // TODO: Matteo. We BLOCK here but most important thing to be doing at this moment.
-      env.getMasterServices().getMasterWalManager().splitMetaLog(serverName);
+      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);
+      }
     }
 
     // Assign meta if still carrying it. Check again: region may be assigned because of RIT timeout
-    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));
+    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);
+        }
+      }
     }
+    return processed;
   }
 
-  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();
+  /**
+   * @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;
       }
+    } catch (InterruptedException ie) {
+      throw new InterruptedIOException("Caught " + ie +
+        " during waitOnRegionToClearRegionsInTransition for " + hri);
     }
-    return !regions.isEmpty();
+    return true;
   }
 
-  private boolean isDefaultMetaRegion(final HRegionInfo hri) {
-    return hri.getTable().equals(TableName.META_TABLE_NAME) &&
-      RegionReplicaUtil.isDefaultReplica(hri);
+  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 void splitLogs(final MasterProcedureEnv env) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Splitting WALs " + this);
+      LOG.debug("Splitting logs from " + serverName + "; region count=" +
+        size(this.regionsOnCrashedServer));
     }
     MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
     AssignmentManager am = env.getMasterServices().getAssignmentManager();
     // TODO: For Matteo. Below BLOCKs!!!! Redo so can relinquish executor while it is running.
-    // PROBLEM!!! WE BLOCK HERE.
     mwm.splitLog(this.serverName);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Done splitting WALs " + this);
-    }
     am.getRegionStates().logSplit(this.serverName);
   }
 
@@ -270,6 +415,124 @@ 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 {
@@ -317,11 +580,11 @@ implements ServerProcedureInterface {
   @Override
   public void toStringClassDetails(StringBuilder sb) {
     sb.append(getClass().getSimpleName());
-    sb.append(" server=");
-    sb.append(serverName);
-    sb.append(", splitWal=");
+    sb.append(" serverName=");
+    sb.append(this.serverName);
+    sb.append(", shouldSplitWal=");
     sb.append(shouldSplitWal);
-    sb.append(", meta=");
+    sb.append(", carryingMeta=");
     sb.append(carryingMeta);
   }
 
@@ -332,6 +595,7 @@ 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()) {
@@ -339,6 +603,11 @@ 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);
   }
 
@@ -349,16 +618,142 @@ 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 ArrayList<HRegionInfo>(size);
+      this.regionsOnCrashedServer = new HashSet<>(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
@@ -394,46 +789,4 @@ implements ServerProcedureInterface {
     // the client does not know about this procedure.
     return false;
   }
-
-  /**
-   * Handle any outstanding RIT that are up against this.serverName, the crashed server.
-   * Notify them of crash. Remove assign entries from the passed in <code>regions</code>
-   * otherwise we have two assigns going on and they will fight over who has lock.
-   * Notify Unassigns also.
-   * @param crashedServer Server that crashed.
-   * @param regions Regions that were on crashed server
-   * @return Subset of <code>regions</code> that were RIT against <code>crashedServer</code>
-   */
-  private void handleRIT(final MasterProcedureEnv env, final List<HRegionInfo> regions) {
-    if (regions == null) return;
-    AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    final Iterator<HRegionInfo> it = regions.iterator();
-    ServerCrashException sce = null;
-    while (it.hasNext()) {
-      final HRegionInfo hri = it.next();
-      RegionTransitionProcedure rtp = am.getRegionStates().getRegionTransitionProcedure(hri);
-      if (rtp == null) continue;
-      // Make sure the RIT is against this crashed server. In the case where there are many
-      // processings of a crashed server -- backed up for whatever reason (slow WAL split) --
-      // then a previous SCP may have already failed an assign, etc., and it may have a new
-      // location target; DO NOT fail these else we make for assign flux.
-      ServerName rtpServerName = rtp.getServer(env);
-      if (rtpServerName == null) {
-        LOG.warn("RIT with ServerName null! " + rtp);
-        continue;
-      }
-      if (!rtpServerName.equals(this.serverName)) continue;
-      LOG.info("pid=" + getProcId() + " found RIT " + rtp + "; " +
-        rtp.getRegionState(env).toShortString());
-      // Notify RIT on server crash.
-      if (sce == null) {
-        sce = new ServerCrashException(getProcId(), getServerName());
-      }
-      rtp.remoteCallFailed(env, this.serverName, sce);
-      if (rtp instanceof AssignProcedure) {
-        // If an assign, include it in our return and remove from passed-in list of regions.
-        it.remove();
-      }
-    }
-  }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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
new file mode 100644
index 0000000..bf9afd7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitTableRegionProcedure.java
@@ -0,0 +1,785 @@
+/**
+ * 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.forNumber(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(), true));
+          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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
index 86143ac..f74df79 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
@@ -31,8 +31,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 public interface TableProcedureInterface {
   public enum TableOperationType {
     CREATE, DELETE, DISABLE, EDIT, ENABLE, READ,
-    REGION_EDIT, REGION_SPLIT, REGION_MERGE, REGION_ASSIGN, REGION_UNASSIGN,
-      REGION_GC, MERGED_REGIONS_GC/* region operations */
+    REGION_EDIT, SPLIT, MERGE, ASSIGN, UNASSIGN, /* region operations */
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 e7f5ead..e41b2cd 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 = env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName());
+          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
           assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
           ProcedureSyncWait.waitRegionInTransition(env, regions);
 
@@ -121,14 +121,12 @@ public class TruncateTableProcedure
           setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
           break;
         case TRUNCATE_TABLE_ASSIGN_REGIONS:
-          CreateTableProcedure.setEnablingState(env, getTableName());
-          addChildProcedure(env.getAssignmentManager().createAssignProcedures(regions));
+          CreateTableProcedure.assignRegions(env, getTableName(), 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/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
index 1ff05eb..25328b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
@@ -118,10 +118,8 @@ public class MobFileCache {
       this.scheduleThreadPool.scheduleAtFixedRate(new EvictionThread(this), period, period,
           TimeUnit.SECONDS);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("MobFileCache enabled with cacheSize=" + mobFileMaxCacheSize +
+      LOG.info("MobFileCache enabled with cacheSize=" + mobFileMaxCacheSize +
           ", evictPeriods=" +  period + "sec, evictRemainRatio=" + evictRemainRatio);
-      }
     } else {
       LOG.info("MobFileCache disabled");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
index 232309b..a30bfef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceAuditor.java
@@ -126,14 +126,14 @@ public class NamespaceAuditor {
     }
   }
 
-  public void updateQuotaForRegionMerge(HRegionInfo mergedRegion) throws IOException {
+  public void updateQuotaForRegionMerge(HRegionInfo hri) throws IOException {
     if (!stateManager.isInitialized()) {
       throw new IOException(
           "Merge operation is being performed even before namespace auditor is initialized.");
-    } else if (!stateManager.checkAndUpdateNamespaceRegionCount(mergedRegion.getTable(),
-        mergedRegion.getRegionName(), -1)) {
-      throw new QuotaExceededException("Region merge not possible for :" +
-        mergedRegion.getEncodedName() + " as quota limits are exceeded ");
+    } else if (!stateManager
+        .checkAndUpdateNamespaceRegionCount(hri.getTable(), hri.getRegionName(), -1)) {
+      throw new QuotaExceededException("Region split not possible for :" + hri.getEncodedName()
+          + " as quota limits are exceeded ");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index 8f6a21d..604f211 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -88,9 +88,8 @@ class NamespaceStateManager {
     if (nspdesc != null) {
       NamespaceTableAndRegionInfo currentStatus;
       currentStatus = getState(namespace);
-      int regionCount = currentStatus.getRegionCount();
-      long maxRegionCount = TableNamespaceManager.getMaxRegions(nspdesc);
-      if (incr > 0 && regionCount >= maxRegionCount) {
+      if (incr > 0 &&
+          currentStatus.getRegionCount() >= TableNamespaceManager.getMaxRegions(nspdesc)) {
         LOG.warn("The region " + Bytes.toStringBinary(regionName)
             + " cannot be created. The region count  will exceed quota on the namespace. "
             + "This may be transient, please retry later if there are any ongoing split"

http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 8f6a33a..9d24e6c 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
@@ -330,7 +330,7 @@ public class MasterQuotaManager implements RegionStateListener {
       namespaceQuotaManager.checkQuotaToCreateTable(tName, regions);
     }
   }
-
+  
   public void checkAndUpdateNamespaceRegionQuota(TableName tName, int regions) throws IOException {
     if (initialized) {
       namespaceQuotaManager.checkQuotaToUpdateRegion(tName, regions);
@@ -347,14 +347,12 @@ public class MasterQuotaManager implements RegionStateListener {
     return -1;
   }
 
-  @Override
-  public void onRegionMerged(HRegionInfo mergedRegion) throws IOException {
+  public void onRegionMerged(HRegionInfo hri) throws IOException {
     if (initialized) {
-      namespaceQuotaManager.updateQuotaForRegionMerge(mergedRegion);
+      namespaceQuotaManager.updateQuotaForRegionMerge(hri);
     }
   }
 
-  @Override
   public void onRegionSplit(HRegionInfo hri) throws IOException {
     if (initialized) {
       namespaceQuotaManager.checkQuotaToSplitRegion(hri);


[22/27] hbase git commit: Revert "HBASE-14614 Procedure v2 - Core Assignment Manager (Matteo Bertozzi)" Revert a mistaken commit!!!

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a3c5a744/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 5ea2044..798932e 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
@@ -842,7 +842,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.TableName table_name = 1;</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> 
+          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<
@@ -960,7 +960,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.ColumnFamilySchema column_families = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>
           getColumnFamiliesFieldBuilder() {
         if (columnFamiliesBuilder_ == null) {
           columnFamiliesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -2257,7 +2257,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.TableName table_name = 1;</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> 
+          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<
@@ -3622,7 +3622,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.TableName table_name = 1;</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> 
+          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<
@@ -3740,7 +3740,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.ColumnFamilySchema column_families = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>
           getColumnFamiliesFieldBuilder() {
         if (columnFamiliesBuilder_ == null) {
           columnFamiliesBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -4955,7 +4955,7 @@ public final class MasterProtos {
        * <code>required .hbase.pb.RegionSpecifier region = 1;</code>
        */
       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> 
+          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.SingleFieldBuilderV3<
@@ -5073,7 +5073,7 @@ public final class MasterProtos {
        * <code>optional .hbase.pb.ServerName dest_server_name = 2;</code>
        */
       private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder> 
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerNameOrBuilder>
           getDestServerNameFieldBuilder() {
         if (destServerNameBuilder_ == null) {
           destServerNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
@@ -5507,35 +5507,33 @@ public final class MasterProtos {
 
   }
 
-  public interface DispatchMergingRegionsRequestOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsRequest)
+  public interface MergeTableRegionsRequestOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsRequest)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
-     */
-    boolean hasRegionA();
-    /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionA();
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>
+        getRegionList();
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionAOrBuilder();
-
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index);
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    boolean hasRegionB();
+    int getRegionCount();
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegionB();
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+        getRegionOrBuilderList();
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionBOrBuilder();
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+        int index);
 
     /**
      * <code>optional bool forcible = 3 [default = false];</code>
@@ -5567,20 +5565,21 @@ public final class MasterProtos {
   /**
    * <pre>
    **
-   * Dispatch merging the specified regions.
+   * Merging the specified regions in a table.
    * </pre>
    *
-   * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
+   * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
    */
-  public  static final class DispatchMergingRegionsRequest extends
+  public  static final class MergeTableRegionsRequest extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@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) {
+      // @@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 DispatchMergingRegionsRequest() {
+    private MergeTableRegionsRequest() {
+      region_ = java.util.Collections.emptyList();
       forcible_ = false;
       nonceGroup_ = 0L;
       nonce_ = 0L;
@@ -5591,7 +5590,7 @@ public final class MasterProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private DispatchMergingRegionsRequest(
+    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 {
@@ -5615,43 +5614,26 @@ public final class MasterProtos {
               break;
             }
             case 10: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = regionA_.toBuilder();
-              }
-              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();
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                region_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier>();
+                mutable_bitField0_ |= 0x00000001;
               }
-              bitField0_ |= 0x00000002;
+              region_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.PARSER, extensionRegistry));
               break;
             }
             case 24: {
-              bitField0_ |= 0x00000004;
+              bitField0_ |= 0x00000001;
               forcible_ = input.readBool();
               break;
             }
             case 32: {
-              bitField0_ |= 0x00000008;
+              bitField0_ |= 0x00000002;
               nonceGroup_ = input.readUInt64();
               break;
             }
             case 40: {
-              bitField0_ |= 0x00000010;
+              bitField0_ |= 0x00000004;
               nonce_ = input.readUInt64();
               break;
             }
@@ -5663,63 +5645,59 @@ 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_DispatchMergingRegionsRequest_descriptor;
+      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_DispatchMergingRegionsRequest_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
     }
 
     private int bitField0_;
-    public static final int REGION_A_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionA_;
-    /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
-     */
-    public boolean hasRegionA() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
+    public static final int REGION_FIELD_NUMBER = 1;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> region_;
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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_;
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier> getRegionList() {
+      return region_;
     }
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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 java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+        getRegionOrBuilderList() {
+      return region_;
     }
-
-    public static final int REGION_B_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier regionB_;
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    public boolean hasRegionB() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+    public int getRegionCount() {
+      return region_.size();
     }
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
      */
-    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_;
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+      return region_.get(index);
     }
     /**
-     * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+     * <code>repeated .hbase.pb.RegionSpecifier region = 1;</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 org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(
+        int index) {
+      return region_.get(index);
     }
 
     public static final int FORCIBLE_FIELD_NUMBER = 3;
@@ -5728,7 +5706,7 @@ public final class MasterProtos {
      * <code>optional bool forcible = 3 [default = false];</code>
      */
     public boolean hasForcible() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
      * <code>optional bool forcible = 3 [default = false];</code>
@@ -5743,7 +5721,7 @@ public final class MasterProtos {
      * <code>optional uint64 nonce_group = 4 [default = 0];</code>
      */
     public boolean hasNonceGroup() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+      return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
      * <code>optional uint64 nonce_group = 4 [default = 0];</code>
@@ -5758,7 +5736,7 @@ public final class MasterProtos {
      * <code>optional uint64 nonce = 5 [default = 0];</code>
      */
     public boolean hasNonce() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
      * <code>optional uint64 nonce = 5 [default = 0];</code>
@@ -5773,21 +5751,11 @@ public final class MasterProtos {
       if (isInitialized == 1) return true;
       if (isInitialized == 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;
+      for (int i = 0; i < getRegionCount(); i++) {
+        if (!getRegion(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
       }
       memoizedIsInitialized = 1;
       return true;
@@ -5795,19 +5763,16 @@ public final class MasterProtos {
 
     public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, getRegionA());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, getRegionB());
+      for (int i = 0; i < region_.size(); i++) {
+        output.writeMessage(1, region_.get(i));
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeBool(3, forcible_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeUInt64(4, nonceGroup_);
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeUInt64(5, nonce_);
       }
       unknownFields.writeTo(output);
@@ -5818,23 +5783,19 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, getRegionA());
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+      for (int i = 0; i < region_.size(); i++) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, getRegionB());
+          .computeMessageSize(1, region_.get(i));
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, forcible_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(4, nonceGroup_);
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeUInt64Size(5, nonce_);
       }
@@ -5849,22 +5810,14 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) obj;
 
       boolean result = true;
-      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 && getRegionList()
+          .equals(other.getRegionList());
       result = result && (hasForcible() == other.hasForcible());
       if (hasForcible()) {
         result = result && (getForcible()
@@ -5891,13 +5844,9 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptor().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 (getRegionCount() > 0) {
+        hash = (37 * hash) + REGION_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionList().hashCode();
       }
       if (hasForcible()) {
         hash = (37 * hash) + FORCIBLE_FIELD_NUMBER;
@@ -5919,58 +5868,58 @@ public final class MasterProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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 {
@@ -5982,7 +5931,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.DispatchMergingRegionsRequest prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -5999,28 +5948,28 @@ public final class MasterProtos {
     /**
      * <pre>
      **
-     * Dispatch merging the specified regions.
+     * Merging the specified regions in a table.
      * </pre>
      *
-     * Protobuf type {@code hbase.pb.DispatchMergingRegionsRequest}
+     * Protobuf type {@code hbase.pb.MergeTableRegionsRequest}
      */
     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.DispatchMergingRegionsRequest)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequestOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsRequest)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequestOrBuilder {
       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_DispatchMergingRegionsRequest_descriptor;
+        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_DispatchMergingRegionsRequest_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -6033,80 +5982,66 @@ public final class MasterProtos {
       private void maybeForceBuilderInitialization() {
         if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
                 .alwaysUseFieldBuilders) {
-          getRegionAFieldBuilder();
-          getRegionBFieldBuilder();
+          getRegionFieldBuilder();
         }
       }
       public Builder clear() {
         super.clear();
-        if (regionABuilder_ == null) {
-          regionA_ = null;
-        } else {
-          regionABuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        if (regionBBuilder_ == null) {
-          regionB_ = null;
+        if (regionBuilder_ == null) {
+          region_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
         } else {
-          regionBBuilder_.clear();
+          regionBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000002);
         forcible_ = false;
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000002);
         nonceGroup_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000004);
         nonce_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000010);
+        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.MasterProtos.internal_static_hbase_pb_DispatchMergingRegionsRequest_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsRequest_descriptor;
       }
 
-      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 getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      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);
+      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);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (regionABuilder_ == null) {
-          result.regionA_ = regionA_;
+        if (regionBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001)) {
+            region_ = java.util.Collections.unmodifiableList(region_);
+            bitField0_ = (bitField0_ & ~0x00000001);
+          }
+          result.region_ = region_;
         } else {
-          result.regionA_ = regionABuilder_.build();
+          result.region_ = regionBuilder_.build();
         }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        if (regionBBuilder_ == null) {
-          result.regionB_ = regionB_;
-        } else {
-          result.regionB_ = regionBBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
+          to_bitField0_ |= 0x00000001;
         }
         result.forcible_ = forcible_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
         }
         result.nonceGroup_ = nonceGroup_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
         }
         result.nonce_ = nonce_;
         result.bitField0_ = to_bitField0_;
@@ -6141,21 +6076,41 @@ 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.DispatchMergingRegionsRequest) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest)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);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      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());
+      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_);
+            }
+          }
         }
         if (other.hasForcible()) {
           setForcible(other.getForcible());
@@ -6172,17 +6127,10 @@ public final class MasterProtos {
       }
 
       public final boolean isInitialized() {
-        if (!hasRegionA()) {
-          return false;
-        }
-        if (!hasRegionB()) {
-          return false;
-        }
-        if (!getRegionA().isInitialized()) {
-          return false;
-        }
-        if (!getRegionB().isInitialized()) {
-          return false;
+        for (int i = 0; i < getRegionCount(); i++) {
+          if (!getRegion(i).isInitialized()) {
+            return false;
+          }
         }
         return true;
       }
@@ -6191,11 +6139,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.DispatchMergingRegionsRequest parsedMessage = null;
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest 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.DispatchMergingRegionsRequest) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest) e.getUnfinishedMessage();
           throw e.unwrapIOException();
         } finally {
           if (parsedMessage != null) {
@@ -6206,240 +6154,244 @@ public final class MasterProtos {
       }
       private int bitField0_;
 
-      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_;
+      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>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public boolean hasRegionA() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+      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>
+       */
+      public int getRegionCount() {
+        if (regionBuilder_ == null) {
+          return region_.size();
+        } else {
+          return regionBuilder_.getCount();
+        }
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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_;
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(int index) {
+        if (regionBuilder_ == null) {
+          return region_.get(index);
         } else {
-          return regionABuilder_.getMessage();
+          return regionBuilder_.getMessage(index);
         }
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegionA(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
-        if (regionABuilder_ == null) {
+      public Builder setRegion(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          regionA_ = value;
+          ensureRegionIsMutable();
+          region_.set(index, value);
           onChanged();
         } else {
-          regionABuilder_.setMessage(value);
+          regionBuilder_.setMessage(index, value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegionA(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionABuilder_ == null) {
-          regionA_ = builderForValue.build();
+      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());
           onChanged();
         } else {
-          regionABuilder_.setMessage(builderForValue.build());
+          regionBuilder_.setMessage(index, builderForValue.build());
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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;
+      public Builder addRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
+        if (regionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
           }
+          ensureRegionIsMutable();
+          region_.add(value);
           onChanged();
         } else {
-          regionABuilder_.mergeFrom(value);
+          regionBuilder_.addMessage(value);
         }
-        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder clearRegionA() {
-        if (regionABuilder_ == null) {
-          regionA_ = null;
+      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);
           onChanged();
         } else {
-          regionABuilder_.clear();
+          regionBuilder_.addMessage(index, value);
         }
-        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
-       */
-      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>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionAOrBuilder() {
-        if (regionABuilder_ != null) {
-          return regionABuilder_.getMessageOrBuilder();
+      public Builder addRegion(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
+        if (regionBuilder_ == null) {
+          ensureRegionIsMutable();
+          region_.add(builderForValue.build());
+          onChanged();
         } else {
-          return regionA_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance() : regionA_;
+          regionBuilder_.addMessage(builderForValue.build());
         }
+        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_a = 1;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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;
+      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());
         }
-        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>required .hbase.pb.RegionSpecifier region_b = 2;</code>
-       */
-      public boolean hasRegionB() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
+        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</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_;
+      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();
         } else {
-          return regionBBuilder_.getMessage();
+          regionBuilder_.addAllMessages(values);
         }
+        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegionB(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier value) {
-        if (regionBBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          regionB_ = value;
+      public Builder clearRegion() {
+        if (regionBuilder_ == null) {
+          region_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000001);
           onChanged();
         } else {
-          regionBBuilder_.setMessage(value);
+          regionBuilder_.clear();
         }
-        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder setRegionB(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) {
-        if (regionBBuilder_ == null) {
-          regionB_ = builderForValue.build();
+      public Builder removeRegion(int index) {
+        if (regionBuilder_ == null) {
+          ensureRegionIsMutable();
+          region_.remove(index);
           onChanged();
         } else {
-          regionBBuilder_.setMessage(builderForValue.build());
+          regionBuilder_.remove(index);
         }
-        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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);
+      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);
         }
-        bitField0_ |= 0x00000002;
-        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public Builder clearRegionB() {
-        if (regionBBuilder_ == null) {
-          regionB_ = null;
-          onChanged();
+      public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>
+           getRegionOrBuilderList() {
+        if (regionBuilder_ != null) {
+          return regionBuilder_.getMessageOrBuilderList();
         } else {
-          regionBBuilder_.clear();
+          return java.util.Collections.unmodifiableList(region_);
         }
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBBuilder() {
-        bitField0_ |= 0x00000002;
-        onChanged();
-        return getRegionBFieldBuilder().getBuilder();
+      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());
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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_;
-        }
+      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());
       }
       /**
-       * <code>required .hbase.pb.RegionSpecifier region_b = 2;</code>
+       * <code>repeated .hbase.pb.RegionSpecifier region = 1;</code>
        */
-      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> 
-          getRegionBFieldBuilder() {
-        if (regionBBuilder_ == null) {
-          regionBBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+      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>(
-                  getRegionB(),
+                  region_,
+                  ((bitField0_ & 0x00000001) == 0x00000001),
                   getParentForChildren(),
                   isClean());
-          regionB_ = null;
+          region_ = null;
         }
-        return regionBBuilder_;
+        return regionBuilder_;
       }
 
       private boolean forcible_ ;
@@ -6447,7 +6399,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public boolean hasForcible() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
        * <code>optional bool forcible = 3 [default = false];</code>
@@ -6459,7 +6411,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public Builder setForcible(boolean value) {
-        bitField0_ |= 0x00000004;
+        bitField0_ |= 0x00000002;
         forcible_ = value;
         onChanged();
         return this;
@@ -6468,7 +6420,7 @@ public final class MasterProtos {
        * <code>optional bool forcible = 3 [default = false];</code>
        */
       public Builder clearForcible() {
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000002);
         forcible_ = false;
         onChanged();
         return this;
@@ -6479,7 +6431,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public boolean hasNonceGroup() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
@@ -6491,7 +6443,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public Builder setNonceGroup(long value) {
-        bitField0_ |= 0x00000008;
+        bitField0_ |= 0x00000004;
         nonceGroup_ = value;
         onChanged();
         return this;
@@ -6500,7 +6452,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce_group = 4 [default = 0];</code>
        */
       public Builder clearNonceGroup() {
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000004);
         nonceGroup_ = 0L;
         onChanged();
         return this;
@@ -6511,7 +6463,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public boolean hasNonce() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
        * <code>optional uint64 nonce = 5 [default = 0];</code>
@@ -6523,7 +6475,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public Builder setNonce(long value) {
-        bitField0_ |= 0x00000010;
+        bitField0_ |= 0x00000008;
         nonce_ = value;
         onChanged();
         return this;
@@ -6532,7 +6484,7 @@ public final class MasterProtos {
        * <code>optional uint64 nonce = 5 [default = 0];</code>
        */
       public Builder clearNonce() {
-        bitField0_ = (bitField0_ & ~0x00000010);
+        bitField0_ = (bitField0_ & ~0x00000008);
         nonce_ = 0L;
         onChanged();
         return this;
@@ -6548,46 +6500,46 @@ public final class MasterProtos {
       }
 
 
-      // @@protoc_insertion_point(builder_scope:hbase.pb.DispatchMergingRegionsRequest)
+      // @@protoc_insertion_point(builder_scope:hbase.pb.MergeTableRegionsRequest)
     }
 
-    // @@protoc_insertion_point(class_scope:hbase.pb.DispatchMergingRegionsRequest)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest DEFAULT_INSTANCE;
+    // @@protoc_insertion_point(class_scope:hbase.pb.MergeTableRegionsRequest)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest DEFAULT_INSTANCE;
     static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest();
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest();
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest getDefaultInstance() {
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstance() {
       return DEFAULT_INSTANCE;
     }
 
-    @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(
+    @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(
           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 DispatchMergingRegionsRequest(input, extensionRegistry);
+          return new MergeTableRegionsRequest(input, extensionRegistry);
       }
     };
 
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsRequest> parser() {
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> parser() {
       return PARSER;
     }
 
     @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<DispatchMergingRegionsRequest> getParserForType() {
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<MergeTableRegionsRequest> getParserForType() {
       return PARSER;
     }
 
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest getDefaultInstanceForType() {
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest getDefaultInstanceForType() {
       return DEFAULT_INSTANCE;
     }
 
   }
 
-  public interface DispatchMergingRegionsResponseOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.DispatchMergingRegionsResponse)
+  public interface MergeTableRegionsResponseOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.MergeTableRegionsResponse)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
@@ -6600,17 +6552,17 @@ public final class MasterProtos {
     long getProcId();
   }
   /**
-   * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
+   * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
    */
-  public  static final class DispatchMergingRegionsResponse extends
+  public  static final class MergeTableRegionsResponse extends
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@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) {
+      // @@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) {
       super(builder);
     }
-    private DispatchMergingRegionsResponse() {
+    private MergeTableRegionsResponse() {
       procId_ = 0L;
     }
 
@@ -6619,7 +6571,7 @@ public final class MasterProtos {
     getUnknownFields() {
       return this.unknownFields;
     }
-    private DispatchMergingRegionsResponse(
+    private MergeTableRegionsResponse(
         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 {
@@ -6661,14 +6613,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_DispatchMergingRegionsResponse_descriptor;
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_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_DispatchMergingRegionsResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.Builder.class);
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
     }
 
     private int bitField0_;
@@ -6725,10 +6677,10 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse) obj;
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse) obj;
 
       boolean result = true;
       result = result && (hasProcId() == other.hasProcId());
@@ -6757,58 +6709,58 @@ public final class MasterProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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.DispatchMergingRegionsResponse parseFrom(
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse 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 {
@@ -6820,7 +6772,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.DispatchMergingRegionsResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse prototype) {
       return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() {
@@ -6835,25 +6787,25 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code hbase.pb.DispatchMergingRegionsResponse}
+     * Protobuf type {@code hbase.pb.MergeTableRegionsResponse}
      */
     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.DispatchMergingRegionsResponse)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponseOrBuilder {
+        // @@protoc_insertion_point(builder_implements:hbase.pb.MergeTableRegionsResponse)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponseOrBuilder {
       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_DispatchMergingRegionsResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_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_DispatchMergingRegionsResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.Builder.class);
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -6877,23 +6829,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_DispatchMergingRegionsResponse_descriptor;
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.internal_static_hbase_pb_MergeTableRegionsResponse_descriptor;
       }
 
-      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 getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse result = buildPartial();
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      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);
+      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);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -6932,2515 +6884,19 @@ 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.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<Dispatc

<TRUNCATED>