You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/08/20 22:16:22 UTC

[1/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Repository: hbase
Updated Branches:
  refs/heads/master 7db116ae1 -> bb3494134


http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java
deleted file mode 100644
index 0e8f97e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestAssignProcedure.java
+++ /dev/null
@@ -1,216 +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.snapshot;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
-import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category({RegionServerTests.class, SmallTests.class})
-public class TestAssignProcedure {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAssignProcedure.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestAssignProcedure.class);
-  @Rule public TestName name = new TestName();
-
-  /**
-   * An override that opens up the updateTransition method inside in AssignProcedure so can call it
-   * below directly in test and mess with targetServer. Used by test
-   * {@link #testTargetServerBeingNulledOnUs()}.
-   */
-  public static class TargetServerBeingNulledOnUsAssignProcedure extends AssignProcedure {
-    public final AtomicBoolean addToRemoteDispatcherWasCalled = new AtomicBoolean(false);
-    public final AtomicBoolean remoteCallFailedWasCalled = new AtomicBoolean(false);
-    private final RegionStates.RegionStateNode rsn;
-
-    public TargetServerBeingNulledOnUsAssignProcedure(RegionInfo regionInfo,
-        RegionStates.RegionStateNode rsn) {
-      super(regionInfo);
-      this.rsn = rsn;
-    }
-
-    /**
-     * Override so can change access from protected to public.
-     */
-    @Override
-    public boolean updateTransition(MasterProcedureEnv env, RegionStates.RegionStateNode regionNode)
-        throws IOException, ProcedureSuspendedException {
-      return super.updateTransition(env, regionNode);
-    }
-
-    @Override
-    protected boolean addToRemoteDispatcher(MasterProcedureEnv env, ServerName targetServer) {
-      // So, mock the ServerCrashProcedure nulling out the targetServer AFTER updateTransition
-      // has been called and BEFORE updateTransition gets to here.
-      // We used to throw a NullPointerException. Now we just say the assign failed so it will
-      // be rescheduled.
-      boolean b = super.addToRemoteDispatcher(env, null);
-      assertFalse(b);
-      // Assert we were actually called.
-      this.addToRemoteDispatcherWasCalled.set(true);
-      return b;
-    }
-
-    @Override
-    public RegionStates.RegionStateNode getRegionState(MasterProcedureEnv env) {
-      // Do this so we don't have to mock a bunch of stuff.
-      return this.rsn;
-    }
-
-    @Override
-    public void remoteCallFailed(final MasterProcedureEnv env,
-        final ServerName serverName, final IOException exception) {
-      // Just skip this remoteCallFailed. Its too hard to mock. Assert it is called though.
-      // Happens after the code we are testing has been called.
-      this.remoteCallFailedWasCalled.set(true);
-    }
-  }
-
-  /**
-   * Test that we deal with ServerCrashProcedure zero'ing out the targetServer in the
-   * RegionStateNode in the midst of our doing an assign. The trickery is done above in
-   * TargetServerBeingNulledOnUsAssignProcedure. We skip a bunch of logic to get at the guts
-   * where the problem happens (We also skip-out the failure handling because it'd take a bunch
-   * of mocking to get it to run). Fix is inside in RemoteProcedureDispatch#addOperationToNode.
-   * It now notices empty targetServer and just returns false so we fall into failure processing
-   * and we'll reassign elsewhere instead of NPE'ing. The fake of ServerCrashProcedure nulling out
-   * the targetServer happens inside in updateTransition just after it was called but before it
-   * gets to the near the end when addToRemoteDispatcher is called. See the
-   * TargetServerBeingNulledOnUsAssignProcedure class above. See HBASE-19218.
-   * Before fix, this test would fail w/ a NullPointerException.
-   */
-  @Test
-  public void testTargetServerBeingNulledOnUs() throws ProcedureSuspendedException, IOException {
-    TableName tn = TableName.valueOf(this.name.getMethodName());
-    RegionInfo ri = RegionInfoBuilder.newBuilder(tn).build();
-    // Create an RSN with location/target server. Will be cleared above in addToRemoteDispatcher to
-    // simulate issue in HBASE-19218
-    RegionStates.RegionStateNode rsn = new RegionStates.RegionStateNode(ri);
-    rsn.setRegionLocation(ServerName.valueOf("server.example.org", 0, 0));
-    MasterProcedureEnv env = Mockito.mock(MasterProcedureEnv.class);
-    AssignmentManager am = Mockito.mock(AssignmentManager.class);
-    ServerManager sm = Mockito.mock(ServerManager.class);
-    Mockito.when(sm.isServerOnline(Mockito.any())).thenReturn(true);
-    MasterServices ms = Mockito.mock(MasterServices.class);
-    Mockito.when(ms.getServerManager()).thenReturn(sm);
-    Configuration configuration = HBaseConfiguration.create();
-    Mockito.when(ms.getConfiguration()).thenReturn(configuration);
-    Mockito.when(env.getAssignmentManager()).thenReturn(am);
-    Mockito.when(env.getMasterServices()).thenReturn(ms);
-    RSProcedureDispatcher rsd = new RSProcedureDispatcher(ms);
-    Mockito.when(env.getRemoteDispatcher()).thenReturn(rsd);
-
-    TargetServerBeingNulledOnUsAssignProcedure assignProcedure =
-        new TargetServerBeingNulledOnUsAssignProcedure(ri, rsn);
-    assignProcedure.updateTransition(env, rsn);
-    assertTrue(assignProcedure.remoteCallFailedWasCalled.get());
-    assertTrue(assignProcedure.addToRemoteDispatcherWasCalled.get());
-  }
-
-  @Test
-  public void testSimpleComparator() {
-    List<AssignProcedure> procedures = new ArrayList<AssignProcedure>();
-    RegionInfo user1 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space1")).build();
-    procedures.add(new AssignProcedure(user1));
-    RegionInfo user2 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space2")).build();
-    procedures.add(new AssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
-    procedures.add(new AssignProcedure(user2));
-    RegionInfo system = RegionInfoBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME).build();
-    procedures.add(new AssignProcedure(system));
-    procedures.sort(AssignProcedure.COMPARATOR);
-    assertTrue(procedures.get(0).isMeta());
-    assertTrue(procedures.get(1).getRegionInfo().getTable().equals(TableName.NAMESPACE_TABLE_NAME));
-  }
-
-  @Test
-  public void testComparatorWithMetas() {
-    List<AssignProcedure> procedures = new ArrayList<AssignProcedure>();
-    RegionInfo user3 = RegionInfoBuilder.newBuilder(TableName.valueOf("user3")).build();
-    procedures.add(new AssignProcedure(user3));
-    RegionInfo system = RegionInfoBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME).build();
-    procedures.add(new AssignProcedure(system));
-    RegionInfo user1 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space1")).build();
-    RegionInfo user2 = RegionInfoBuilder.newBuilder(TableName.valueOf("user_space2")).build();
-    procedures.add(new AssignProcedure(user1));
-    RegionInfo meta2 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-        setStartKey(Bytes.toBytes("002")).build();
-    procedures.add(new AssignProcedure(meta2));
-    procedures.add(new AssignProcedure(user2));
-    RegionInfo meta1 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-        setStartKey(Bytes.toBytes("001")).build();
-    procedures.add(new AssignProcedure(meta1));
-    procedures.add(new AssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
-    RegionInfo meta0 = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
-        setStartKey(Bytes.toBytes("000")).build();
-    procedures.add(new AssignProcedure(meta0));
-    for (int i = 0; i < 10; i++) {
-      Collections.shuffle(procedures);
-      procedures.sort(AssignProcedure.COMPARATOR);
-      try {
-        assertTrue(procedures.get(0).getRegionInfo().equals(RegionInfoBuilder.FIRST_META_REGIONINFO));
-        assertTrue(procedures.get(1).getRegionInfo().equals(meta0));
-        assertTrue(procedures.get(2).getRegionInfo().equals(meta1));
-        assertTrue(procedures.get(3).getRegionInfo().equals(meta2));
-        assertTrue(procedures.get(4).getRegionInfo().getTable().equals(TableName.NAMESPACE_TABLE_NAME));
-        assertTrue(procedures.get(5).getRegionInfo().equals(user1));
-        assertTrue(procedures.get(6).getRegionInfo().equals(user2));
-        assertTrue(procedures.get(7).getRegionInfo().equals(user3));
-      } catch (Throwable t) {
-        for (AssignProcedure proc : procedures) {
-          LOG.debug(Objects.toString(proc));
-        }
-        throw t;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMove.java
index 86c1e61..badd8db 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionMove.java
@@ -23,12 +23,11 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.List;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 import org.apache.hadoop.hbase.client.Put;
@@ -132,11 +131,13 @@ public class TestRegionMove {
     // Disable the table
     admin.disableTable(tableName);
 
-    // We except a DNRIOE when we try to move a region which isn't open.
-    thrown.expect(TableNotEnabledException.class);
-    thrown.expectMessage(t.getName().toString());
-
-    // Move the region to the other RS -- should fail
-    admin.move(regionToMove.getEncodedNameAsBytes(), Bytes.toBytes(rs2.getServerName().toString()));
+    try {
+      // Move the region to the other RS -- should fail
+      admin.move(regionToMove.getEncodedNameAsBytes(),
+        Bytes.toBytes(rs2.getServerName().toString()));
+      fail();
+    } catch (DoNotRetryIOException e) {
+      // We got expected exception
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 eb162de..b9a1e4a 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
@@ -33,15 +33,14 @@ import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
@@ -52,6 +51,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
@@ -64,12 +64,13 @@ 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.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TestReplicasClient.SlowMeCopro;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
 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.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
@@ -77,14 +78,11 @@ import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 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.RegionStateNode;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
-import org.apache.hadoop.hbase.shaded.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;
@@ -94,8 +92,6 @@ import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
-import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.junit.After;
@@ -111,11 +107,18 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
+import org.apache.hbase.thirdparty.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;
+
 /**
  * The below tests are testing split region against a running cluster
  */
 @Category({RegionServerTests.class, LargeTests.class})
-@SuppressWarnings("deprecation")
 public class TestSplitTransactionOnCluster {
 
   @ClassRule
@@ -151,7 +154,7 @@ public class TestSplitTransactionOnCluster {
   @After
   public void tearDown() throws Exception {
     this.admin.close();
-    for (HTableDescriptor htd: this.admin.listTables()) {
+    for (TableDescriptor htd: this.admin.listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TESTING_UTIL.deleteTable(htd.getTableName());
     }
@@ -192,7 +195,7 @@ public class TestSplitTransactionOnCluster {
       t.close();
 
       // Turn off balancer so it doesn't cut in and mess up our placements.
-      this.admin.setBalancerRunning(false, true);
+      this.admin.balancerSwitch(false, true);
       // Turn off the meta scanner so it don't remove parent on us.
       master.setCatalogJanitorEnabled(false);
 
@@ -207,7 +210,7 @@ public class TestSplitTransactionOnCluster {
         master.getConfiguration());
 
       // split async
-      this.admin.splitRegion(region.getRegionInfo().getRegionName(), new byte[] {42});
+      this.admin.splitRegionAsync(region.getRegionInfo().getRegionName(), new byte[] { 42 });
 
       // we have to wait until the SPLITTING state is seen by the master
       FailingSplitMasterObserver observer =
@@ -221,7 +224,7 @@ public class TestSplitTransactionOnCluster {
       }
       assertTrue(cluster.getMaster().getAssignmentManager().getRegionStates().isRegionOnline(hri));
     } finally {
-      admin.setBalancerRunning(true, false);
+      admin.balancerSwitch(true, false);
       master.setCatalogJanitorEnabled(true);
       abortAndWaitForMaster();
       TESTING_UTIL.deleteTable(tableName);
@@ -232,9 +235,9 @@ public class TestSplitTransactionOnCluster {
   public void testSplitFailedCompactionAndSplit() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     // Create table then get the single region for our new table.
-    HTableDescriptor htd = new HTableDescriptor(tableName);
     byte[] cf = Bytes.toBytes("cf");
-    htd.addFamily(new HColumnDescriptor(cf));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf)).build();
     admin.createTable(htd);
 
     for (int i = 0; cluster.getRegions(tableName).isEmpty() && i < 100; i++) {
@@ -309,7 +312,7 @@ public class TestSplitTransactionOnCluster {
     RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
 
     // Turn off balancer so it doesn't cut in and mess up our placements.
-    this.admin.setBalancerRunning(false, true);
+    this.admin.balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     cluster.getMaster().setCatalogJanitorEnabled(false);
     try {
@@ -326,7 +329,7 @@ public class TestSplitTransactionOnCluster {
       // We don't roll back here anymore. Instead we fail-fast on construction of the
       // split transaction. Catch the exception instead.
       try {
-        this.admin.splitRegion(hri.getRegionName());
+        this.admin.splitRegionAsync(hri.getRegionName(), null);
         fail();
       } catch (DoNotRetryRegionException e) {
         // Expected
@@ -343,7 +346,7 @@ public class TestSplitTransactionOnCluster {
       checkAndGetDaughters(tableName);
       // OK, so split happened after we cleared the blocking node.
     } finally {
-      admin.setBalancerRunning(true, false);
+      admin.balancerSwitch(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
     }
@@ -367,7 +370,7 @@ public class TestSplitTransactionOnCluster {
     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
 
     // Turn off balancer so it doesn't cut in and mess up our placements.
-    this.admin.setBalancerRunning(false, true);
+    this.admin.balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     cluster.getMaster().setCatalogJanitorEnabled(false);
     try {
@@ -405,7 +408,7 @@ public class TestSplitTransactionOnCluster {
         }
       }
       assertTrue(daughterRegion != null);
-      for (int i=0; i<100; i++) {
+      for (int i = 0; i < 100; i++) {
         if (!daughterRegion.hasReferences()) break;
         Threads.sleep(100);
       }
@@ -437,7 +440,7 @@ public class TestSplitTransactionOnCluster {
       }
     } finally {
       LOG.info("EXITING");
-      admin.setBalancerRunning(true, false);
+      admin.balancerSwitch(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
     }
@@ -446,9 +449,8 @@ public class TestSplitTransactionOnCluster {
   @Test
   public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
     TableName userTableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor htd = new HTableDescriptor(userTableName);
-    HColumnDescriptor hcd = new HColumnDescriptor("col");
-    htd.addFamily(hcd);
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(userTableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("col")).build();
     admin.createTable(htd);
     Table table = TESTING_UTIL.getConnection().getTable(userTableName);
     try {
@@ -480,7 +482,7 @@ public class TestSplitTransactionOnCluster {
       p.addColumn("col".getBytes(), "ql".getBytes(), "val".getBytes());
       table.put(p);
       admin.flush(userTableName);
-      admin.splitRegion(hRegionInfo.getRegionName(), "row7".getBytes());
+      admin.splitRegionAsync(hRegionInfo.getRegionName(), "row7".getBytes());
       regionsOfTable = cluster.getMaster()
           .getAssignmentManager().getRegionStates()
           .getRegionsOfTable(userTableName);
@@ -530,7 +532,7 @@ public class TestSplitTransactionOnCluster {
     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
 
     // Turn off balancer so it doesn't cut in and mess up our placements.
-    this.admin.setBalancerRunning(false, true);
+    this.admin.balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     cluster.getMaster().setCatalogJanitorEnabled(false);
     try {
@@ -540,7 +542,7 @@ public class TestSplitTransactionOnCluster {
       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
       printOutRegions(server, "Initial regions: ");
       // Call split.
-      this.admin.splitRegion(hri.getRegionName());
+      this.admin.splitRegionAsync(hri.getRegionName(), null);
       List<HRegion> daughters = checkAndGetDaughters(tableName);
 
       // Before cleanup, get a new master.
@@ -575,7 +577,7 @@ public class TestSplitTransactionOnCluster {
       ServerName regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
       assertEquals(null, regionServerOfRegion);
     } finally {
-      TESTING_UTIL.getAdmin().setBalancerRunning(true, false);
+      TESTING_UTIL.getAdmin().balancerSwitch(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
     }
@@ -601,7 +603,7 @@ public class TestSplitTransactionOnCluster {
       HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
       insertData(tableName, admin, t);
       // Turn off balancer so it doesn't cut in and mess up our placements.
-      admin.setBalancerRunning(false, true);
+      admin.balancerSwitch(false, true);
       // Turn off the meta scanner so it don't remove parent on us.
       cluster.getMaster().setCatalogJanitorEnabled(false);
       boolean tableExists = MetaTableAccessor.tableExists(regionServer.getConnection(),
@@ -648,7 +650,7 @@ public class TestSplitTransactionOnCluster {
       SlowMeCopro.getPrimaryCdl().get().countDown();
     } finally {
       SlowMeCopro.getPrimaryCdl().get().countDown();
-      admin.setBalancerRunning(true, false);
+      admin.balancerSwitch(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
       t.close();
     }
@@ -676,8 +678,7 @@ public class TestSplitTransactionOnCluster {
    * into two regions with no store files.
    */
   @Test
-  public void testSplitRegionWithNoStoreFiles()
-      throws Exception {
+  public void testSplitRegionWithNoStoreFiles() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     // Create table then get the single region for our new table.
     createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@@ -688,7 +689,7 @@ public class TestSplitTransactionOnCluster {
       .getRegionName());
     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
     // Turn off balancer so it doesn't cut in and mess up our placements.
-    this.admin.setBalancerRunning(false, true);
+    this.admin.balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     cluster.getMaster().setCatalogJanitorEnabled(false);
     try {
@@ -740,23 +741,25 @@ public class TestSplitTransactionOnCluster {
       assertTrue(regionStates.isRegionInState(daughters.get(1).getRegionInfo(), State.OPEN));
 
       // We should not be able to assign it again
-      am.assign(hri);
-      assertFalse("Split region can't be assigned",
-        regionStates.isRegionInTransition(hri));
+      try {
+        am.assign(hri);
+      } catch (DoNotRetryIOException e) {
+        // Expected
+      }
+      assertFalse("Split region can't be assigned", regionStates.isRegionInTransition(hri));
       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) {
+      } catch (DoNotRetryIOException e) {
         // Expected
       }
-      assertFalse("Split region can't be unassigned",
-        regionStates.isRegionInTransition(hri));
+      assertFalse("Split region can't be unassigned", regionStates.isRegionInTransition(hri));
       assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
     } finally {
-      admin.setBalancerRunning(true, false);
+      admin.balancerSwitch(true, false);
       cluster.getMaster().setCatalogJanitorEnabled(true);
     }
   }
@@ -766,21 +769,23 @@ public class TestSplitTransactionOnCluster {
       throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      htd.addFamily(new HColumnDescriptor("f"));
-      htd.addFamily(new HColumnDescriptor("i_f"));
-      htd.setRegionSplitPolicyClassName(CustomSplitPolicy.class.getName());
+      byte[] cf = Bytes.toBytes("f");
+      byte[] cf1 = Bytes.toBytes("i_f");
+      TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf1))
+        .setRegionSplitPolicyClassName(CustomSplitPolicy.class.getName()).build();
       admin.createTable(htd);
       List<HRegion> regions = awaitTableRegions(tableName);
       HRegion region = regions.get(0);
       for(int i = 3;i<9;i++) {
         Put p = new Put(Bytes.toBytes("row"+i));
-        p.addColumn(Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("value" + i));
-        p.addColumn(Bytes.toBytes("i_f"), Bytes.toBytes("q"), Bytes.toBytes("value" + i));
+        p.addColumn(cf, Bytes.toBytes("q"), Bytes.toBytes("value" + i));
+        p.addColumn(cf1, Bytes.toBytes("q"), Bytes.toBytes("value" + i));
         region.put(p);
       }
       region.flush(true);
-      HStore store = region.getStore(Bytes.toBytes("f"));
+      HStore store = region.getStore(cf);
       Collection<HStoreFile> storefiles = store.getStorefiles();
       assertEquals(1, storefiles.size());
       assertFalse(region.hasReferences());
@@ -833,7 +838,7 @@ public class TestSplitTransactionOnCluster {
 
   private void split(final RegionInfo hri, final HRegionServer server, final int regionCount)
       throws IOException, InterruptedException {
-    admin.splitRegion(hri.getRegionName());
+    admin.splitRegionAsync(hri.getRegionName(), null);
     for (int i = 0; cluster.getRegions(hri.getTable()).size() <= regionCount && i < 60; i++) {
       LOG.debug("Waiting on region " + hri.getRegionNameAsString() + " to split");
       Thread.sleep(2000);
@@ -999,7 +1004,7 @@ 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:
+        for (RegionStateNode regionState:
           regionStates.getRegionsInTransition()) {
           /* TODO!!!!
           // Find the merging_new region and remove it


[7/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region


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

Branch: refs/heads/master
Commit: bb3494134edb6a4e607ab199e4d0542135d83a64
Parents: 7db116a
Author: zhangduo <zh...@apache.org>
Authored: Mon Aug 20 21:02:56 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Tue Aug 21 06:12:09 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/master/RegionState.java | 180 ++---
 .../hbase/procedure2/ProcedureExecutor.java     |  29 +-
 .../hbase/procedure2/StateMachineProcedure.java |   2 +-
 .../procedure2/ProcedureTestingUtility.java     |  29 +-
 .../src/main/protobuf/ClusterStatus.proto       |   4 +
 .../src/main/protobuf/MasterProcedure.proto     |  28 +
 .../hbase/rsgroup/RSGroupAdminServer.java       |   2 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |   2 +-
 .../hadoop/hbase/master/AssignmentListener.java |  42 --
 .../org/apache/hadoop/hbase/master/HMaster.java |  88 ++-
 .../hadoop/hbase/master/MasterDumpServlet.java  |   2 +-
 .../master/assignment/AssignProcedure.java      | 290 +------
 .../master/assignment/AssignmentManager.java    | 752 +++++++++----------
 .../assignment/AssignmentManagerUtil.java       | 195 +++++
 .../master/assignment/CloseRegionProcedure.java |  82 ++
 .../assignment/MergeTableRegionsProcedure.java  | 125 ++-
 .../master/assignment/MoveRegionProcedure.java  |  69 +-
 .../master/assignment/OpenRegionProcedure.java  |  67 ++
 .../assignment/RegionRemoteProcedureBase.java   | 157 ++++
 .../master/assignment/RegionStateNode.java      | 313 ++++++++
 .../master/assignment/RegionStateStore.java     |   2 +-
 .../hbase/master/assignment/RegionStates.java   | 472 ++----------
 .../assignment/RegionTransitionProcedure.java   | 347 +--------
 .../hbase/master/assignment/ServerState.java    |  55 ++
 .../master/assignment/ServerStateNode.java      | 128 ++++
 .../assignment/SplitTableRegionProcedure.java   | 123 ++-
 .../assignment/TransitRegionStateProcedure.java | 569 ++++++++++++++
 .../master/assignment/UnassignProcedure.java    | 238 +-----
 .../hadoop/hbase/master/assignment/Util.java    |  72 --
 .../AbstractStateMachineRegionProcedure.java    |   7 +-
 .../AbstractStateMachineTableProcedure.java     |  27 +-
 .../master/procedure/CreateTableProcedure.java  |   2 +-
 .../master/procedure/DeleteTableProcedure.java  |  12 +-
 .../master/procedure/DisableTableProcedure.java |  53 +-
 .../master/procedure/EnableTableProcedure.java  |   3 +-
 .../master/procedure/InitMetaProcedure.java     |   7 +-
 .../master/procedure/RecoverMetaProcedure.java  | 199 +----
 .../procedure/ReopenTableRegionsProcedure.java  |  35 +-
 .../master/procedure/ServerCrashProcedure.java  | 146 ++--
 .../procedure/TruncateTableProcedure.java       |   7 +-
 .../hbase/master/TestAssignmentListener.java    | 294 --------
 .../master/TestMasterAbortAndRSGotKilled.java   |  67 +-
 .../TestMergeTableRegionsWhileRSCrash.java      |   9 +-
 ...stServerCrashProcedureCarryingMetaStuck.java |   9 +-
 .../master/TestServerCrashProcedureStuck.java   |   9 +-
 .../master/TestSplitRegionWhileRSCrash.java     |   7 +-
 .../master/assignment/MockMasterServices.java   |   2 +-
 .../assignment/TestAMAssignWithRandExec.java    |  53 ++
 .../assignment/TestAMServerFailedOpen.java      | 134 ++++
 .../assignment/TestAssignmentManager.java       | 699 +----------------
 .../assignment/TestAssignmentManagerBase.java   | 586 +++++++++++++++
 .../assignment/TestAssignmentManagerUtil.java   | 134 ++++
 .../assignment/TestCloseRegionWhileRSCrash.java | 237 ++++++
 .../TestMergeTableRegionsProcedure.java         |  32 +-
 .../TestSplitTableRegionProcedure.java          |   9 +-
 .../TestTransitRegionStateProcedure.java        | 164 ++++
 .../TestUnexpectedStateException.java           | 167 ----
 .../TestFavoredStochasticLoadBalancer.java      |   2 +-
 .../MasterProcedureTestingUtility.java          |  24 +-
 .../procedure/TestCloneSnapshotProcedure.java   |  29 +-
 .../procedure/TestEnableTableProcedure.java     |   3 +-
 .../procedure/TestRecoverMetaProcedure.java     | 109 ---
 .../procedure/TestServerCrashProcedure.java     |   4 +
 .../procedure/TestTruncateTableProcedure.java   |   5 +-
 .../master/snapshot/TestAssignProcedure.java    | 216 ------
 .../hbase/regionserver/TestRegionMove.java      |  17 +-
 .../TestSplitTransactionOnCluster.java          | 105 +--
 67 files changed, 4004 insertions(+), 4084 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 7289ce8..745e1ea 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
@@ -51,9 +51,13 @@ public class RegionState {
     SPLITTING_NEW,  // new region to be created when RS splits a parent
                     // region but hasn't be created yet, or master doesn't
                     // know it's already created
-    MERGING_NEW;    // new region to be created when RS merges two
+    MERGING_NEW,    // new region to be created when RS merges two
                     // daughter regions but hasn't be created yet, or
                     // master doesn't know it's already created
+    ABNORMALLY_CLOSED; // the region is CLOSED because of a RS crashes. Usually it is the same
+                       // with CLOSED, but for some operations such as merge/split, we can not
+                       // apply it to a region in this state, as it may lead to data loss as we
+                       // may have some data in recovered edits.
 
     /**
      * Convert to protobuf ClusterStatusProtos.RegionState.State
@@ -61,47 +65,50 @@ public class RegionState {
     public ClusterStatusProtos.RegionState.State convert() {
       ClusterStatusProtos.RegionState.State rs;
       switch (this) {
-      case OFFLINE:
-        rs = ClusterStatusProtos.RegionState.State.OFFLINE;
-        break;
-      case OPENING:
-        rs = ClusterStatusProtos.RegionState.State.OPENING;
-        break;
-      case OPEN:
-        rs = ClusterStatusProtos.RegionState.State.OPEN;
-        break;
-      case CLOSING:
-        rs = ClusterStatusProtos.RegionState.State.CLOSING;
-        break;
-      case CLOSED:
-        rs = ClusterStatusProtos.RegionState.State.CLOSED;
-        break;
-      case SPLITTING:
-        rs = ClusterStatusProtos.RegionState.State.SPLITTING;
-        break;
-      case SPLIT:
-        rs = ClusterStatusProtos.RegionState.State.SPLIT;
-        break;
-      case FAILED_OPEN:
-        rs = ClusterStatusProtos.RegionState.State.FAILED_OPEN;
-        break;
-      case FAILED_CLOSE:
-        rs = ClusterStatusProtos.RegionState.State.FAILED_CLOSE;
-        break;
-      case MERGING:
-        rs = ClusterStatusProtos.RegionState.State.MERGING;
-        break;
-      case MERGED:
-        rs = ClusterStatusProtos.RegionState.State.MERGED;
-        break;
-      case SPLITTING_NEW:
-        rs = ClusterStatusProtos.RegionState.State.SPLITTING_NEW;
-        break;
-      case MERGING_NEW:
-        rs = ClusterStatusProtos.RegionState.State.MERGING_NEW;
-        break;
-      default:
-        throw new IllegalStateException("");
+        case OFFLINE:
+          rs = ClusterStatusProtos.RegionState.State.OFFLINE;
+          break;
+        case OPENING:
+          rs = ClusterStatusProtos.RegionState.State.OPENING;
+          break;
+        case OPEN:
+          rs = ClusterStatusProtos.RegionState.State.OPEN;
+          break;
+        case CLOSING:
+          rs = ClusterStatusProtos.RegionState.State.CLOSING;
+          break;
+        case CLOSED:
+          rs = ClusterStatusProtos.RegionState.State.CLOSED;
+          break;
+        case SPLITTING:
+          rs = ClusterStatusProtos.RegionState.State.SPLITTING;
+          break;
+        case SPLIT:
+          rs = ClusterStatusProtos.RegionState.State.SPLIT;
+          break;
+        case FAILED_OPEN:
+          rs = ClusterStatusProtos.RegionState.State.FAILED_OPEN;
+          break;
+        case FAILED_CLOSE:
+          rs = ClusterStatusProtos.RegionState.State.FAILED_CLOSE;
+          break;
+        case MERGING:
+          rs = ClusterStatusProtos.RegionState.State.MERGING;
+          break;
+        case MERGED:
+          rs = ClusterStatusProtos.RegionState.State.MERGED;
+          break;
+        case SPLITTING_NEW:
+          rs = ClusterStatusProtos.RegionState.State.SPLITTING_NEW;
+          break;
+        case MERGING_NEW:
+          rs = ClusterStatusProtos.RegionState.State.MERGING_NEW;
+          break;
+        case ABNORMALLY_CLOSED:
+          rs = ClusterStatusProtos.RegionState.State.ABNORMALLY_CLOSED;
+          break;
+        default:
+          throw new IllegalStateException("");
       }
       return rs;
     }
@@ -114,49 +121,52 @@ public class RegionState {
     public static State convert(ClusterStatusProtos.RegionState.State protoState) {
       State state;
       switch (protoState) {
-      case OFFLINE:
-        state = OFFLINE;
-        break;
-      case PENDING_OPEN:
-      case OPENING:
-        state = OPENING;
-        break;
-      case OPEN:
-        state = OPEN;
-        break;
-      case PENDING_CLOSE:
-      case CLOSING:
-        state = CLOSING;
-        break;
-      case CLOSED:
-        state = CLOSED;
-        break;
-      case SPLITTING:
-        state = SPLITTING;
-        break;
-      case SPLIT:
-        state = SPLIT;
-        break;
-      case FAILED_OPEN:
-        state = FAILED_OPEN;
-        break;
-      case FAILED_CLOSE:
-        state = FAILED_CLOSE;
-        break;
-      case MERGING:
-        state = MERGING;
-        break;
-      case MERGED:
-        state = MERGED;
-        break;
-      case SPLITTING_NEW:
-        state = SPLITTING_NEW;
-        break;
-      case MERGING_NEW:
-        state = MERGING_NEW;
-        break;
-      default:
-        throw new IllegalStateException("Unhandled state " + protoState);
+        case OFFLINE:
+          state = OFFLINE;
+          break;
+        case PENDING_OPEN:
+        case OPENING:
+          state = OPENING;
+          break;
+        case OPEN:
+          state = OPEN;
+          break;
+        case PENDING_CLOSE:
+        case CLOSING:
+          state = CLOSING;
+          break;
+        case CLOSED:
+          state = CLOSED;
+          break;
+        case SPLITTING:
+          state = SPLITTING;
+          break;
+        case SPLIT:
+          state = SPLIT;
+          break;
+        case FAILED_OPEN:
+          state = FAILED_OPEN;
+          break;
+        case FAILED_CLOSE:
+          state = FAILED_CLOSE;
+          break;
+        case MERGING:
+          state = MERGING;
+          break;
+        case MERGED:
+          state = MERGED;
+          break;
+        case SPLITTING_NEW:
+          state = SPLITTING_NEW;
+          break;
+        case MERGING_NEW:
+          state = MERGING_NEW;
+          break;
+        case ABNORMALLY_CLOSED:
+          state = ABNORMALLY_CLOSED;
+          break;
+        default:
+          throw new IllegalStateException("Unhandled state " + protoState);
       }
       return state;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 695c7b0..fe97404 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
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -94,6 +95,7 @@ public class ProcedureExecutor<TEnvironment> {
    * Class with parameters describing how to fail/die when in testing-context.
    */
   public static class Testing {
+    protected boolean killIfHasParent = true;
     protected boolean killIfSuspended = false;
 
     /**
@@ -120,8 +122,14 @@ public class ProcedureExecutor<TEnvironment> {
       return kill;
     }
 
-    protected boolean shouldKillBeforeStoreUpdate(final boolean isSuspended) {
-      return (isSuspended && !killIfSuspended) ? false : shouldKillBeforeStoreUpdate();
+    protected boolean shouldKillBeforeStoreUpdate(boolean isSuspended, boolean hasParent) {
+      if (isSuspended && !killIfSuspended) {
+        return false;
+      }
+      if (hasParent && !killIfHasParent) {
+        return false;
+      }
+      return shouldKillBeforeStoreUpdate();
     }
 
     protected boolean shouldKillAfterStoreUpdate() {
@@ -457,6 +465,7 @@ public class ProcedureExecutor<TEnvironment> {
     int failedCount = 0;
     while (procIter.hasNext()) {
       boolean finished = procIter.isNextFinished();
+      @SuppressWarnings("unchecked")
       Procedure<TEnvironment> proc = procIter.next();
       NonceKey nonceKey = proc.getNonceKey();
       long procId = proc.getProcId();
@@ -508,6 +517,7 @@ public class ProcedureExecutor<TEnvironment> {
         continue;
       }
 
+      @SuppressWarnings("unchecked")
       Procedure<TEnvironment> proc = procIter.next();
       assert !(proc.isFinished() && !proc.hasParent()) : "unexpected completed proc=" + proc;
 
@@ -1180,6 +1190,17 @@ public class ProcedureExecutor<TEnvironment> {
   }
 
   /**
+   * Should only be used when starting up, where the procedure workers have not been started.
+   * <p/>
+   * If the procedure works has been started, the return values maybe changed when you are
+   * processing it so usually this is not safe. Use {@link #getProcedures()} below for most cases as
+   * it will do a copy, and also include the finished procedures.
+   */
+  public Collection<Procedure<TEnvironment>> getActiveProceduresNoCopy() {
+    return procedures.values();
+  }
+
+  /**
    * Get procedures.
    * @return the procedures in a list
    */
@@ -1607,7 +1628,8 @@ public class ProcedureExecutor<TEnvironment> {
 
       // allows to kill the executor before something is stored to the wal.
       // useful to test the procedure recovery.
-      if (testing != null && testing.shouldKillBeforeStoreUpdate(suspended)) {
+      if (testing != null &&
+        testing.shouldKillBeforeStoreUpdate(suspended, procedure.hasParent())) {
         kill("TESTING: Kill BEFORE store update: " + procedure);
       }
 
@@ -1839,6 +1861,7 @@ public class ProcedureExecutor<TEnvironment> {
       long lastUpdate = EnvironmentEdgeManager.currentTime();
       try {
         while (isRunning() && keepAlive(lastUpdate)) {
+          @SuppressWarnings("unchecked")
           Procedure<TEnvironment> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
           if (proc == null) {
             continue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 4ed82f2..986b250 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
@@ -185,7 +185,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
         this.cycles++;
       }
 
-      LOG.trace("{}", toString());
+      LOG.trace("{}", this);
       stateFlow = executeFromState(env, state);
       if (!hasMoreState()) setNextState(EOF_STATE);
       if (subProcList != null && !subProcList.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 138215b..d52b6bb 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -26,7 +26,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Set;
 import java.util.concurrent.Callable;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -37,14 +36,16 @@ import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-import org.apache.hbase.thirdparty.com.google.protobuf.BytesValue;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.hbase.util.Threads;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.BytesValue;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 public class ProcedureTestingUtility {
   private static final Logger LOG = LoggerFactory.getLogger(ProcedureTestingUtility.class);
 
@@ -67,7 +68,7 @@ public class ProcedureTestingUtility {
   }
 
   public static <TEnv> void restart(final ProcedureExecutor<TEnv> procExecutor) throws Exception {
-    restart(procExecutor, false, true, null, null);
+    restart(procExecutor, false, true, null, null, null);
   }
 
   public static void initAndStartWorkers(ProcedureExecutor<?> procExecutor, int numThreads,
@@ -76,9 +77,9 @@ public class ProcedureTestingUtility {
     procExecutor.startWorkers();
   }
 
-  public static <TEnv> void restart(final ProcedureExecutor<TEnv> procExecutor,
-      final boolean avoidTestKillDuringRestart, final boolean failOnCorrupted,
-      final Callable<Void> stopAction, final Callable<Void> startAction)
+  public static <TEnv> void restart(ProcedureExecutor<TEnv> procExecutor,
+      boolean avoidTestKillDuringRestart, boolean failOnCorrupted, Callable<Void> stopAction,
+      Callable<Void> actionBeforeStartWorker, Callable<Void> startAction)
       throws Exception {
     final ProcedureStore procStore = procExecutor.getStore();
     final int storeThreads = procExecutor.getCorePoolSize();
@@ -104,7 +105,11 @@ public class ProcedureTestingUtility {
     // re-start
     LOG.info("RESTART - Start");
     procStore.start(storeThreads);
-    initAndStartWorkers(procExecutor, execThreads, failOnCorrupted);
+    procExecutor.init(execThreads, failOnCorrupted);
+    if (actionBeforeStartWorker != null) {
+      actionBeforeStartWorker.call();
+    }
+    procExecutor.startWorkers();
     if (startAction != null) {
       startAction.call();
     }
@@ -139,6 +144,12 @@ public class ProcedureTestingUtility {
     }
   }
 
+  public static <TEnv> void setKillIfHasParent(ProcedureExecutor<TEnv> procExecutor,
+      boolean value) {
+    createExecutorTesting(procExecutor);
+    procExecutor.testing.killIfHasParent = value;
+  }
+
   public static <TEnv> void setKillIfSuspended(ProcedureExecutor<TEnv> procExecutor,
       boolean value) {
     createExecutorTesting(procExecutor);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
index 399ff5e..d39db36 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ClusterStatus.proto
@@ -52,6 +52,10 @@ message RegionState {
     MERGING_NEW = 14;  // new region to be created when RS merges two
                        // daughter regions but hasn't be created yet, or
                        // master doesn't know it's already created
+    ABNORMALLY_CLOSED = 15;// the region is CLOSED because of a RS crash. Usually it is the same
+                       // with CLOSED, but for some operations such as merge/split, we can not
+                       // apply it to a region in this state, as it may lead to data loss as we
+                       // may have some data in recovered edits.
   }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 5227e64..e50a913 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -519,3 +519,31 @@ message ReplaySyncReplicationWALParameter {
   required string peer_id = 1;
   repeated string wal = 2;
 }
+
+enum RegionStateTransitionState {
+  REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE = 1;
+  REGION_STATE_TRANSITION_OPEN = 2;
+  REGION_STATE_TRANSITION_CONFIRM_OPENED = 3;
+  REGION_STATE_TRANSITION_CLOSE = 4;
+  REGION_STATE_TRANSITION_CONFIRM_CLOSED = 5;
+}
+
+message RegionStateTransitionStateData {
+  required RegionStateTransitionState initialState = 1;
+  required RegionStateTransitionState lastState = 2;
+  optional ServerName assign_candidate = 3;
+  required bool force_new_plan = 4;
+}
+
+message RegionRemoteProcedureBaseStateData {
+  required RegionInfo region = 1;
+  required ServerName target_server = 2;
+  required bool dispatched = 3;
+}
+
+message OpenRegionProcedureStateData {
+}
+
+message CloseRegionProcedureStateData {
+  optional ServerName assign_candidate = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 b39d3a1..720b193 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
@@ -41,7 +41,7 @@ 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.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 8e70f5e..ee0651b 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.master.MasterServices;
 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.RegionStateNode;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
deleted file mode 100644
index 84a7042..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentListener.java
+++ /dev/null
@@ -1,42 +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.ServerName;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Get notification of assignment events. The invocations are inline
- * so make sure your implementation is fast else you'll slow hbase.
- */
-@InterfaceAudience.Private
-public interface AssignmentListener {
-  /**
-   * The region was opened on the specified server.
-   * @param regionInfo The opened region.
-   * @param serverName The remote servers name.
-   */
-  void regionOpened(final RegionInfo regionInfo, final ServerName serverName);
-
-  /**
-   * The region was closed on the region server.
-   * @param regionInfo The closed region.
-   */
-  void regionClosed(final RegionInfo regionInfo);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 abea148..50794f4 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
@@ -100,10 +100,14 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
+import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.MoveRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
+import org.apache.hadoop.hbase.master.assignment.UnassignProcedure;
 import org.apache.hadoop.hbase.master.balancer.BalancerChore;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
@@ -130,6 +134,7 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
 import org.apache.hadoop.hbase.master.replication.AbstractPeerProcedure;
@@ -213,6 +218,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
@@ -797,6 +803,45 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.mpmHost.initialize(this, this.metricsMaster);
   }
 
+  private static final ImmutableSet<Class<?>> UNSUPPORTED_PROCEDURES =
+    ImmutableSet.of(RecoverMetaProcedure.class, AssignProcedure.class, UnassignProcedure.class,
+      MoveRegionProcedure.class);
+
+  /**
+   * In HBASE-20811, we have introduced a new TRSP to assign/unassign/move regions, and it is
+   * incompatible with the old AssignProcedure/UnassignProcedure/MoveRegionProcedure. So we need to
+   * make sure that there are none these procedures when upgrading. If there are, the master will
+   * quit, you need to go back to the old version to finish these procedures first before upgrading.
+   */
+  private void checkUnsupportedProcedure(
+      Map<Class<? extends Procedure>, List<Procedure<MasterProcedureEnv>>> procsByType)
+      throws HBaseIOException {
+    // Confirm that we do not have unfinished assign/unassign related procedures. It is not easy to
+    // support both the old assign/unassign procedures and the new TransitRegionStateProcedure as
+    // there will be conflict in the code for AM. We should finish all these procedures before
+    // upgrading.
+    for (Class<?> clazz : UNSUPPORTED_PROCEDURES) {
+      List<Procedure<MasterProcedureEnv>> procs = procsByType.get(clazz);
+      if (procs != null) {
+        LOG.error(
+          "Unsupported procedure type {} found, please rollback your master to the old" +
+            " version to finish them, and then try to upgrade again. The full procedure list: {}",
+          clazz, procs);
+        throw new HBaseIOException("Unsupported procedure type " + clazz + " found");
+      }
+    }
+    // A special check for SCP, as we do not support RecoverMetaProcedure any more so we need to
+    // make sure that no one will try to schedule it but SCP does have a state which will schedule
+    // it.
+    if (procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
+      .map(p -> (ServerCrashProcedure) p).anyMatch(ServerCrashProcedure::isInRecoverMetaState)) {
+      LOG.error("At least one ServerCrashProcedure is going to schedule a RecoverMetaProcedure," +
+        " which is not supported any more. Please rollback your master to the old version to" +
+        " finish them, and then try to upgrade again.");
+      throw new HBaseIOException("Unsupported procedure state found for ServerCrashProcedure");
+    }
+  }
+
   /**
    * Finish initialization of HMaster after becoming the primary master.
    * <p/>
@@ -870,24 +915,39 @@ public class HMaster extends HRegionServer implements MasterServices {
     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
     this.clusterId = clusterId.toString();
 
-
-
     status.setStatus("Initialze ServerManager and schedule SCP for crash servers");
     this.serverManager = createServerManager(this);
     createProcedureExecutor();
+    @SuppressWarnings("rawtypes")
+    Map<Class<? extends Procedure>, List<Procedure<MasterProcedureEnv>>> procsByType =
+      procedureExecutor.getActiveProceduresNoCopy().stream()
+        .collect(Collectors.groupingBy(p -> p.getClass()));
+
+    checkUnsupportedProcedure(procsByType);
+
     // Create Assignment Manager
     this.assignmentManager = new AssignmentManager(this);
     this.assignmentManager.start();
+    // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
+    // completed, it could still be in the procedure list. This is a bit strange but is another
+    // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.
+    List<TransitRegionStateProcedure> ritList =
+      procsByType.getOrDefault(TransitRegionStateProcedure.class, Collections.emptyList()).stream()
+        .filter(p -> !p.isFinished()).map(p -> (TransitRegionStateProcedure) p)
+        .collect(Collectors.toList());
+    this.assignmentManager.setupRIT(ritList);
+
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start(
-      procedureExecutor.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
-        .map(p -> ((ServerCrashProcedure) p).getServerName()).collect(Collectors.toSet()),
+      procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
+        .map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()),
       walManager.getLiveServersFromWALDir());
     // This manager will be started AFTER hbase:meta is confirmed on line.
     // hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table
     // state from zookeeper while hbase2 reads it from hbase:meta. Disable if no hbase1 clients.
     this.tableStateManager =
-        this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, true)?
+      this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, true)
+        ?
         new MirroringTableStateManager(this):
         new TableStateManager(this);
 
@@ -3577,7 +3637,6 @@ public class HMaster extends HRegionServer implements MasterServices {
    * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
    * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
    * @param servers Region servers to decommission.
-   * @throws HBaseIOException
    */
   public void decommissionRegionServers(final List<ServerName> servers, final boolean offload)
       throws HBaseIOException {
@@ -3590,7 +3649,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         ZKUtil.createAndFailSilent(getZooKeeper(), node);
       } catch (KeeperException ke) {
         throw new HBaseIOException(
-            this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);
+          this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);
       }
       if (this.serverManager.addServerToDrainList(server)) {
         serversAdded.add(server);
@@ -3601,7 +3660,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       final List<ServerName> destServers = this.serverManager.createDestinationServersList();
       for (ServerName server : serversAdded) {
         final List<RegionInfo> regionsOnServer =
-            this.assignmentManager.getRegionStates().getServerRegionInfoSet(server);
+          this.assignmentManager.getRegionStates().getServerRegionInfoSet(server);
         for (RegionInfo hri : regionsOnServer) {
           ServerName dest = balancer.randomAssignment(hri, destServers);
           if (dest == null) {
@@ -3627,10 +3686,9 @@ public class HMaster extends HRegionServer implements MasterServices {
    * Remove decommission marker (previously called 'draining') from a region server to allow regions
    * assignments. Load regions onto the server asynchronously if a list of regions is given
    * @param server Region server to remove decommission marker from.
-   * @throws HBaseIOException
    */
   public void recommissionRegionServer(final ServerName server,
-      final List<byte[]> encodedRegionNames) throws HBaseIOException {
+      final List<byte[]> encodedRegionNames) throws IOException {
     // Remove the server from decommissioned (draining) server list.
     String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
     String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
@@ -3638,7 +3696,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);
     } catch (KeeperException ke) {
       throw new HBaseIOException(
-          this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);
+        this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);
     }
     this.serverManager.removeServerFromDrainList(server);
 
@@ -3651,15 +3709,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     for (byte[] encodedRegionName : encodedRegionNames) {
       RegionState regionState =
-          assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
+        assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
       if (regionState == null) {
         LOG.warn("Unknown region " + Bytes.toStringBinary(encodedRegionName));
         continue;
       }
       RegionInfo hri = regionState.getRegion();
       if (server.equals(regionState.getServerName())) {
-        LOG.info("Skipping move of region " + hri.getRegionNameAsString()
-          + " because region already assigned to the same server " + server + ".");
+        LOG.info("Skipping move of region " + hri.getRegionNameAsString() +
+          " because region already assigned to the same server " + server + ".");
         continue;
       }
       RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), server);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 0dd50ff..ec8e523 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
@@ -30,7 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerMetrics;
 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.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.monitoring.LogMonitoring;
 import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 55aee4a..33a3545 100644
--- 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
@@ -1,5 +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
@@ -16,20 +15,11 @@
  * 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.Comparator;
-
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfo;
-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.TableStateManager;
-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.ProcedureMetrics;
@@ -37,75 +27,27 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 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?
+ * Leave here only for checking if we can successfully start the master.
+ * @deprecated Do not use any more.
+ * @see TransitRegionStateProcedure
  */
 // TODO: Add being able to assign a region to open read-only.
+@Deprecated
 @InterfaceAudience.Private
 public class AssignProcedure extends RegionTransitionProcedure {
-  private static final Logger LOG = LoggerFactory.getLogger(AssignProcedure.class);
 
-  /**
-   * Set to true when we need recalibrate -- choose a new target -- because original assign failed.
-   */
   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;
 
-  /**
-   * Comparator that will sort AssignProcedures so meta assigns come first, then system table
-   * assigns and finally user space assigns.
-   */
-  public static final CompareAssignProcedure COMPARATOR = new CompareAssignProcedure();
-
   public AssignProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  public AssignProcedure(final RegionInfo regionInfo) {
-    super(regionInfo);
-    this.targetServer = null;
-  }
-
-  public AssignProcedure(final RegionInfo regionInfo, final ServerName destinationServer) {
-    super(regionInfo);
-    this.targetServer = destinationServer;
   }
 
   @Override
@@ -124,10 +66,9 @@ public class AssignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
-    final AssignRegionStateData.Builder state = AssignRegionStateData.newBuilder()
-        .setTransitionState(getTransitionState())
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    final AssignRegionStateData.Builder state =
+      AssignRegionStateData.newBuilder().setTransitionState(getTransitionState())
         .setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
     if (forceNewPlan) {
       state.setForceNewPlan(true);
@@ -142,8 +83,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     final AssignRegionStateData state = serializer.deserialize(AssignRegionStateData.class);
     setTransitionState(state.getTransitionState());
     setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
@@ -159,202 +99,36 @@ public class AssignProcedure extends RegionTransitionProcedure {
   @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;
-    }
-    // Don't assign if table is in disabling or disabled state.
-    TableStateManager tsm = env.getMasterServices().getTableStateManager();
-    TableName tn = regionNode.getRegionInfo().getTable();
-    if (tsm.getTableState(tn).isDisabledOrDisabling()) {
-      LOG.info("Table " + tn + " state=" + tsm.getTableState(tn) + ", skipping " + this);
-      return false;
-    }
-    // If the region is SPLIT, we can't assign it. But state might be CLOSED, rather than
-    // SPLIT which is what a region gets set to when unassigned as part of SPLIT. FIX.
-    if (regionNode.isInState(State.SPLIT) ||
-        (regionNode.getRegionInfo().isOffline() && regionNode.getRegionInfo().isSplit())) {
-      LOG.info("SPLIT, cannot be assigned; " + this + "; " + regionNode +
-        "; hri=" + regionNode.getRegionInfo());
-      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). We call regionNode.offline below to set state to
-    // OFFLINE and to clear the 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.
-    // TODO: Region will be set into OFFLINE state below regardless of what its previous state was
-    // This is dangerous? Wrong? What if region was in an unexpected state?
-    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);
-        } else if (regionNode.getLastHost() != null) {
-          retain = true;
-          LOG.info("Setting lastHost as the region location " + regionNode.getLastHost());
-          regionNode.setRegionLocation(regionNode.getLastHost());
-        }
-      }
-    }
-    LOG.info("Starting " + 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.
+      throws IOException, ProcedureSuspendedException {
     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()) {
-          // Don't bother logging if openSeqNum == 0
-          if (openSeqNum != 0) {
-            LOG.warn("Skipping update of open seqnum with " + openSeqNum +
-                " because current seqnum=" + regionNode.getOpenSeqNum());
-          }
-        } else {
-          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) {
+  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);
+      env.getAssignmentManager().getFavoredNodes(getRegionInfo()), false);
   }
 
   @Override
   protected boolean remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
       final IOException exception) {
-    handleFailure(env, regionNode);
     return true;
   }
 
@@ -365,43 +139,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  public ServerName getServer(final MasterProcedureEnv env) {
-    RegionStateNode node =
-        env.getAssignmentManager().getRegionStates().getRegionStateNode(this.getRegionInfo());
-    if (node == null) return null;
-    return node.getRegionLocation();
-  }
-
-  @Override
   protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
     return env.getAssignmentManager().getAssignmentManagerMetrics().getAssignProcMetrics();
   }
-
-  /**
-   * Sort AssignProcedures such that meta and system assigns come first before user-space assigns.
-   * Have to do it this way w/ distinct Comparator because Procedure is already Comparable on
-   * 'Env'(?).
-   */
-  public static class CompareAssignProcedure implements Comparator<AssignProcedure> {
-    @Override
-    public int compare(AssignProcedure left, AssignProcedure right) {
-      if (left.getRegionInfo().isMetaRegion()) {
-        if (right.getRegionInfo().isMetaRegion()) {
-          return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
-        }
-        return -1;
-      } else if (right.getRegionInfo().isMetaRegion()) {
-        return +1;
-      }
-      if (left.getRegionInfo().getTable().isSystemTable()) {
-        if (right.getRegionInfo().getTable().isSystemTable()) {
-          return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
-        }
-        return -1;
-      } else if (right.getRegionInfo().getTable().isSystemTable()) {
-        return +1;
-      }
-      return RegionInfo.COMPARATOR.compare(left.getRegionInfo(), right.getRegionInfo());
-    }
-  }
 }


[5/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
new file mode 100644
index 0000000..6770e68
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
@@ -0,0 +1,157 @@
+/**
+ * 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.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionRemoteProcedureBaseStateData;
+
+/**
+ * The base class for the remote procedures used to open/close a region.
+ * <p/>
+ * Notice that here we do not care about the result of the remote call, if the remote call is
+ * finished, either succeeded or not, we will always finish the procedure. The parent procedure
+ * should take care of the result and try to reschedule if the result is not good.
+ */
+@InterfaceAudience.Private
+public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedureEnv>
+    implements TableProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RegionRemoteProcedureBase.class);
+
+  protected RegionInfo region;
+
+  private ServerName targetServer;
+
+  private boolean dispatched;
+
+  protected RegionRemoteProcedureBase() {
+  }
+
+  protected RegionRemoteProcedureBase(RegionInfo region, ServerName targetServer) {
+    this.region = region;
+    this.targetServer = targetServer;
+  }
+
+  @Override
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    // should not be called since we use reportRegionStateTransition to report the result
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) {
+    // should not be called since we use reportRegionStateTransition to report the result
+    throw new UnsupportedOperationException();
+  }
+
+  private ProcedureEvent<?> getRegionEvent(MasterProcedureEnv env) {
+    return env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(region)
+      .getProcedureEvent();
+  }
+
+  @Override
+  public void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
+      IOException exception) {
+    ProcedureEvent<?> event = getRegionEvent(env);
+    synchronized (event) {
+      if (event.isReady()) {
+        LOG.warn(
+          "The procedure event of procedure {} for region {} to server {} is not suspended, " +
+            "usually this should not happen, but anyway let's skip the following wake up code, ",
+          this, region, targetServer);
+        return;
+      }
+      LOG.warn("The remote operation {} for region {} to server {} failed", this, region,
+        targetServer, exception);
+      event.wake(env.getProcedureScheduler());
+    }
+  }
+
+  @Override
+  public TableName getTableName() {
+    return region.getTable();
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    return false;
+  }
+
+  @Override
+  protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+      throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+    if (dispatched) {
+      // we are done, the parent procedure will check whether we are succeeded.
+      return null;
+    }
+    ProcedureEvent<?> event = getRegionEvent(env);
+    synchronized (event) {
+      try {
+        env.getRemoteDispatcher().addOperationToNode(targetServer, this);
+      } catch (FailedRemoteDispatchException e) {
+        LOG.warn("Can not add remote operation {} for region {} to server {}, this usually " +
+          "because the server is alread dead, give up and mark the procedure as complete, " +
+          "the parent procedure will take care of this.", this, region, targetServer, e);
+        return null;
+      }
+      dispatched = true;
+      event.suspend();
+      event.suspendIfNotReady(this);
+      throw new ProcedureSuspendedException();
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    serializer.serialize(RegionRemoteProcedureBaseStateData.newBuilder()
+      .setRegion(ProtobufUtil.toRegionInfo(region))
+      .setTargetServer(ProtobufUtil.toServerName(targetServer)).setDispatched(dispatched).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    RegionRemoteProcedureBaseStateData data =
+      serializer.deserialize(RegionRemoteProcedureBaseStateData.class);
+    region = ProtobufUtil.toRegionInfo(data.getRegion());
+    targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+    dispatched = data.getDispatched();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
new file mode 100644
index 0000000..81e6f78
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateNode.java
@@ -0,0 +1,313 @@
+/**
+ * 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.Arrays;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionOfflineException;
+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.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Current Region State. Most fields are synchronized with meta region, i.e, we will update meta
+ * immediately after we modify this RegionStateNode, and usually under the lock. The only exception
+ * is {@link #lastHost}, which should not be used for critical condition.
+ * <p/>
+ * Typically, the only way to modify this class is through {@link TransitRegionStateProcedure}, and
+ * we will record the TRSP along with this RegionStateNode to make sure that there could at most one
+ * TRSP. For other operations, such as SCP, we will first get the lock, and then try to schedule a
+ * TRSP. If there is already one, then the solution will be different:
+ * <ul>
+ * <li>For SCP, we will update the region state in meta to tell the TRSP to retry.</li>
+ * <li>For DisableTableProcedure, as we have the xlock, we can make sure that the TRSP has not been
+ * executed yet, so just unset it and attach a new one. The original one will quit immediately when
+ * executing.</li>
+ * <li>For split/merge, we will fail immediately as there is no actual operations yet so no
+ * harm.</li>
+ * <li>For EnableTableProcedure/TruncateTableProcedure, we can make sure that there will be no TRSP
+ * attached with the RSNs.</li>
+ * <li>For other procedures, you'd better use ReopenTableRegionsProcedure. The RTRP will take care
+ * of lots of corner cases when reopening regions.</li>
+ * </ul>
+ * <p/>
+ * Several fields are declared with {@code volatile}, which means you are free to get it without
+ * lock, but usually you should not use these fields without locking for critical condition, as it
+ * will be easily to introduce inconsistency. For example, you are free to dump the status and show
+ * it on web without locking, but if you want to change the state of the RegionStateNode by checking
+ * the current state, you'd better have the lock...
+ */
+@InterfaceAudience.Private
+public class RegionStateNode implements Comparable<RegionStateNode> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RegionStateNode.class);
+
+  private static final class AssignmentProcedureEvent extends ProcedureEvent<RegionInfo> {
+    public AssignmentProcedureEvent(final RegionInfo regionInfo) {
+      super(regionInfo);
+    }
+  }
+
+  @VisibleForTesting
+  final Lock lock = new ReentrantLock();
+  private final RegionInfo regionInfo;
+  private final ProcedureEvent<?> event;
+  private final ConcurrentMap<RegionInfo, RegionStateNode> ritMap;
+
+  // volatile only for getLastUpdate and test usage, the upper layer should sync on the
+  // RegionStateNode before accessing usually.
+  private volatile TransitRegionStateProcedure procedure = null;
+  private volatile ServerName regionLocation = null;
+  // notice that, the lastHost will only be updated when a region is successfully CLOSED through
+  // UnassignProcedure, so do not use it for critical condition as the data maybe stale and unsync
+  // with the data in meta.
+  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;
+
+  RegionStateNode(RegionInfo regionInfo, ConcurrentMap<RegionInfo, RegionStateNode> ritMap) {
+    this.regionInfo = regionInfo;
+    this.event = new AssignmentProcedureEvent(regionInfo);
+    this.ritMap = ritMap;
+  }
+
+  /**
+   * @param update new region state this node should be assigned.
+   * @param expected current state should be in this given list of expected states
+   * @return true, if current state is in expected list; otherwise false.
+   */
+  public boolean setState(final State update, final State... expected) {
+    if (!isInState(expected)) {
+      return false;
+    }
+    this.state = update;
+    this.lastUpdate = EnvironmentEdgeManager.currentTime();
+    return true;
+  }
+
+  /**
+   * 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 void 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());
+    }
+  }
+
+  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() {
+    TransitRegionStateProcedure proc = this.procedure;
+    return proc != null ? proc.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 void setProcedure(TransitRegionStateProcedure proc) {
+    assert this.procedure == null;
+    this.procedure = proc;
+    ritMap.put(regionInfo, this);
+  }
+
+  public void unsetProcedure(TransitRegionStateProcedure proc) {
+    assert this.procedure == proc;
+    this.procedure = null;
+    ritMap.remove(regionInfo, this);
+  }
+
+  public TransitRegionStateProcedure getProcedure() {
+    return procedure;
+  }
+
+  public ProcedureEvent<?> getProcedureEvent() {
+    return event;
+  }
+
+  public RegionInfo 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;
+  }
+
+  public RegionState toRegionState() {
+    return new RegionState(getRegionInfo(), getState(), getLastUpdate(), getRegionLocation());
+  }
+
+  @Override
+  public int compareTo(final RegionStateNode other) {
+    // NOTE: RegionInfo sort by table first, so we are relying on that.
+    // we have a TestRegionState#testOrderedByTable() that check for that.
+    return RegionInfo.COMPARATOR.compare(getRegionInfo(), 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());
+  }
+
+  public void checkOnline() throws DoNotRetryRegionException {
+    RegionInfo ri = getRegionInfo();
+    State s = state;
+    if (s != State.OPEN) {
+      throw new DoNotRetryRegionException(ri.getEncodedName() + " is no OPEN; state=" + s);
+    }
+    if (ri.isSplitParent()) {
+      throw new DoNotRetryRegionException(
+        ri.getEncodedName() + " is not online (splitParent=true)");
+    }
+    if (ri.isSplit()) {
+      throw new DoNotRetryRegionException(ri.getEncodedName() + " has split=true");
+    }
+    if (ri.isOffline()) {
+      // RegionOfflineException is not instance of DNRIOE so wrap it.
+      throw new DoNotRetryRegionException(new RegionOfflineException(ri.getEncodedName()));
+    }
+  }
+
+  public void lock() {
+    lock.lock();
+  }
+
+  public void unlock() {
+    lock.unlock();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index aeef835..48ec4fb 100644
--- 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
@@ -127,7 +127,7 @@ public class RegionStateStore {
     }
   }
 
-  public void updateRegionLocation(RegionStates.RegionStateNode regionStateNode)
+  public void updateRegionLocation(RegionStateNode regionStateNode)
       throws IOException {
     if (regionStateNode.getRegionInfo().isMetaRegion()) {
       updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 9f01293..26a6884 100644
--- 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
@@ -1,5 +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
@@ -16,11 +15,9 @@
  * 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;
@@ -28,7 +25,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
@@ -41,12 +37,9 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
-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 org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,247 +55,22 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 public class RegionStates {
   private static final Logger LOG = LoggerFactory.getLogger(RegionStates.class);
 
-  protected static final State[] STATES_EXPECTED_ON_OPEN = new State[] {
+  // TODO: need to be more specific, i.e, OPENING vs. OPEN, CLOSING vs. CLOSED.
+  static final State[] STATES_EXPECTED_ON_OPEN = new State[] {
     State.OPEN, // State may already be OPEN if we died after receiving the OPEN from regionserver
                 // but before complete finish of AssignProcedure. HBASE-20100.
-    State.OFFLINE, State.CLOSED,      // disable/offline
-    State.SPLITTING, State.SPLIT,     // ServerCrashProcedure
+    State.OFFLINE, State.CLOSED, State.ABNORMALLY_CLOSED, // disable/offline
+    State.SPLITTING,     // ServerCrashProcedure
     State.OPENING, State.FAILED_OPEN, // already in-progress (retrying)
+    State.MERGED, State.SPLITTING_NEW
   };
 
-  protected static final State[] STATES_EXPECTED_ON_CLOSE = new State[] {
-    State.SPLITTING, State.SPLIT, State.MERGING, // ServerCrashProcedure
+  static final State[] STATES_EXPECTED_ON_CLOSE = new State[] {
+    State.SPLITTING, State.MERGING, State.OPENING, // ServerCrashProcedure
     State.OPEN,                   // enabled/open
     State.CLOSING                 // already in-progress (retrying)
   };
 
-  private static class AssignmentProcedureEvent extends ProcedureEvent<RegionInfo> {
-    public AssignmentProcedureEvent(final RegionInfo 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 RegionInfo regionInfo;
-    private final ProcedureEvent<?> event;
-
-    private volatile RegionTransitionProcedure procedure = null;
-    private volatile ServerName regionLocation = null;
-    // notice that, the lastHost will only be updated when a region is successfully CLOSED through
-    // UnassignProcedure, so do not use it for critical condition as the data maybe stale and unsync
-    // with the data in meta.
-    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 RegionInfo regionInfo) {
-      this.regionInfo = regionInfo;
-      this.event = new AssignmentProcedureEvent(regionInfo);
-    }
-
-    /**
-     * @param update new region state this node should be assigned.
-     * @param expected current state should be in this given list of expected states
-     * @return true, if current state is in expected list; otherwise false.
-     */
-    public boolean setState(final State update, final State... expected) {
-      if (!isInState(expected)) {
-        return false;
-      }
-      this.state = update;
-      this.lastUpdate = EnvironmentEdgeManager.currentTime();
-      return true;
-    }
-
-    /**
-     * 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 void 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());
-      }
-    }
-
-    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 RegionInfo 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;
-    }
-
-    public RegionState toRegionState() {
-      return new RegionState(getRegionInfo(), getState(), getLastUpdate(), getRegionLocation());
-    }
-
-    @Override
-    public int compareTo(final RegionStateNode other) {
-      // NOTE: RegionInfo sort by table first, so we are relying on that.
-      // we have a TestRegionState#testOrderedByTable() that check for that.
-      return RegionInfo.COMPARATOR.compare(getRegionInfo(), 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.
@@ -314,130 +82,6 @@ public class RegionStates {
     }
   }
 
-  /**
-   * Server State.
-   */
-  public enum ServerState {
-    /**
-     * Initial state. Available.
-     */
-    ONLINE,
-
-    /**
-     * Only server which carries meta can have this state. We will split wal for meta and then
-     * assign meta first before splitting other wals.
-     */
-    SPLITTING_META,
-
-    /**
-     * Indicate that the meta splitting is done. We need this state so that the UnassignProcedure
-     * for meta can safely quit. See the comments in UnassignProcedure.remoteCallFailed for more
-     * details.
-     */
-    SPLITTING_META_DONE,
-
-    /**
-     * Server expired/crashed. Currently undergoing WAL splitting.
-     */
-    SPLITTING,
-
-    /**
-     * WAL splitting done. This state will be used to tell the UnassignProcedure that it can safely
-     * quit. See the comments in UnassignProcedure.remoteCallFailed for more details.
-     */
-    OFFLINE
-  }
-
-  /**
-   * State of Server; list of hosted regions, etc.
-   */
-  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;
-
-    public ServerStateNode(final ServerName serverName) {
-      this.serverName = serverName;
-      this.regions = ConcurrentHashMap.newKeySet();
-      this.reportEvent = new ServerReportEvent(serverName);
-    }
-
-    public ServerName getServerName() {
-      return serverName;
-    }
-
-    public ServerState getState() {
-      return state;
-    }
-
-    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;
-    }
-
-    private void setState(final ServerState state) {
-      this.state = state;
-    }
-
-    public Set<RegionStateNode> getRegions() {
-      return regions;
-    }
-
-    public int getRegionCount() {
-      return regions.size();
-    }
-
-    public ArrayList<RegionInfo> getRegionInfoList() {
-      ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(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();
 
@@ -482,22 +126,23 @@ public class RegionStates {
   // ==========================================================================
   //  RegionStateNode helpers
   // ==========================================================================
-  protected RegionStateNode createRegionStateNode(final RegionInfo regionInfo) {
-    RegionStateNode newNode = new RegionStateNode(regionInfo);
+  @VisibleForTesting
+  RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
+    RegionStateNode newNode = new RegionStateNode(regionInfo, regionInTransition);
     RegionStateNode oldNode = regionsMap.putIfAbsent(regionInfo.getRegionName(), newNode);
     return oldNode != null ? oldNode : newNode;
   }
 
-  protected RegionStateNode getOrCreateRegionStateNode(final RegionInfo regionInfo) {
-    RegionStateNode node = regionsMap.get(regionInfo.getRegionName());
+  public RegionStateNode getOrCreateRegionStateNode(RegionInfo regionInfo) {
+    RegionStateNode node = getRegionStateNodeFromName(regionInfo.getRegionName());
     return node != null ? node : createRegionStateNode(regionInfo);
   }
 
-  RegionStateNode getRegionStateNodeFromName(final byte[] regionName) {
+  RegionStateNode getRegionStateNodeFromName(byte[] regionName) {
     return regionsMap.get(regionName);
   }
 
-  protected RegionStateNode getRegionStateNode(final RegionInfo regionInfo) {
+  public RegionStateNode getRegionStateNode(RegionInfo regionInfo) {
     return getRegionStateNodeFromName(regionInfo.getRegionName());
   }
 
@@ -593,7 +238,8 @@ public class RegionStates {
   }
 
   private HRegionLocation createRegionForReopen(RegionStateNode node) {
-    synchronized (node) {
+    node.lock();
+    try {
       if (!include(node, false)) {
         return null;
       }
@@ -605,6 +251,8 @@ public class RegionStates {
       } else {
         return null;
       }
+    } finally {
+      node.unlock();
     }
   }
 
@@ -649,7 +297,8 @@ public class RegionStates {
     if (node == null) {
       return null;
     }
-    synchronized (node) {
+    node.lock();
+    try {
       if (oldLoc.getSeqNum() >= 0) {
         // in OPEN state before
         if (node.isInState(State.OPEN)) {
@@ -683,6 +332,8 @@ public class RegionStates {
           return new HRegionLocation(node.getRegionInfo(), node.getRegionLocation(), openSeqNum);
         }
       }
+    } finally {
+      node.unlock();
     }
   }
 
@@ -726,8 +377,10 @@ public class RegionStates {
    * @return set of RegionInfo hosted by the specified server
    */
   public List<RegionInfo> getServerRegionInfoSet(final ServerName serverName) {
-    final ServerStateNode serverInfo = getServerNode(serverName);
-    if (serverInfo == null) return Collections.emptyList();
+    ServerStateNode serverInfo = getServerNode(serverName);
+    if (serverInfo == null) {
+      return Collections.emptyList();
+    }
 
     synchronized (serverInfo) {
       return serverInfo.getRegionInfoList();
@@ -779,10 +432,13 @@ public class RegionStates {
     setServerState(serverName, ServerState.OFFLINE);
   }
 
-  public void updateRegionState(final RegionInfo regionInfo, final State state) {
-    final RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo);
-    synchronized (regionNode) {
+  public void updateRegionState(RegionInfo regionInfo, State state) {
+    RegionStateNode regionNode = getOrCreateRegionStateNode(regionInfo);
+    regionNode.lock();
+    try {
       regionNode.setState(state);
+    } finally {
+      regionNode.unlock();
     }
   }
 
@@ -799,11 +455,14 @@ public class RegionStates {
     return result;
   }
 
-  public boolean isRegionInState(final RegionInfo regionInfo, final State... state) {
-    final RegionStateNode region = getRegionStateNode(regionInfo);
-    if (region != null) {
-      synchronized (region) {
-        return region.isInState(state);
+  public boolean isRegionInState(RegionInfo regionInfo, State... state) {
+    RegionStateNode regionNode = getRegionStateNode(regionInfo);
+    if (regionNode != null) {
+      regionNode.lock();
+      try {
+        return regionNode.isInState(state);
+      } finally {
+        regionNode.unlock();
       }
     }
     return false;
@@ -866,12 +525,15 @@ public class RegionStates {
     return tableRegions;
   }
 
-  public ServerName getRegionServerOfRegion(final RegionInfo regionInfo) {
-    final RegionStateNode region = getRegionStateNode(regionInfo);
-    if (region != null) {
-      synchronized (region) {
-        ServerName server = region.getRegionLocation();
-        return server != null ? server : region.getLastHost();
+  public ServerName getRegionServerOfRegion(RegionInfo regionInfo) {
+    RegionStateNode regionNode = getRegionStateNode(regionInfo);
+    if (regionNode != null) {
+      regionNode.lock();
+      try {
+        ServerName server = regionNode.getRegionLocation();
+        return server != null ? server : regionNode.getLastHost();
+      } finally {
+        regionNode.unlock();
       }
     }
     return null;
@@ -938,20 +600,6 @@ public class RegionStates {
   // ==========================================================================
   //  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();
   }
@@ -961,21 +609,17 @@ public class RegionStates {
     return node != null ? node.isInTransition() : false;
   }
 
-  /**
-   * @return If a procedure-in-transition for <code>hri</code>, return it else null.
-   */
-  public RegionTransitionProcedure getRegionTransitionProcedure(final RegionInfo hri) {
+  public RegionState getRegionTransitionState(RegionInfo hri) {
     RegionStateNode node = regionInTransition.get(hri);
-    if (node == null) return null;
-    return node.getProcedure();
-  }
-
-  public RegionState getRegionTransitionState(final RegionInfo hri) {
-    RegionStateNode node = regionInTransition.get(hri);
-    if (node == null) return null;
+    if (node == null) {
+      return null;
+    }
 
-    synchronized (node) {
+    node.lock();
+    try {
       return node.isInTransition() ? node.toRegionState() : null;
+    } finally {
+      node.unlock();
     }
   }
 
@@ -1110,7 +754,7 @@ public class RegionStates {
     serverMap.remove(serverName);
   }
 
-  protected ServerStateNode getServerNode(final ServerName serverName) {
+  ServerStateNode getServerNode(final ServerName serverName) {
     return serverMap.get(serverName);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 0db8676..2f94765 100644
--- 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
@@ -1,5 +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
@@ -24,100 +23,41 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 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.FailedRemoteDispatchException;
 import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 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.procedure2.RemoteProcedureException;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 
 /**
- * Base class for the Assign and Unassign Procedure.
- *
- * Locking:
- * Takes exclusive lock on the region being assigned/unassigned. Thus, there can only be one
- * RegionTransitionProcedure per region running at a time (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>
- *
- * <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.</p>
- *
- * <p>NOTE: {@link AssignProcedure} and {@link UnassignProcedure} should not be thought of
- * as being asymmetric, at least currently.
- * <ul>
- * <li>{@link AssignProcedure} moves through all the above described states and implements methods
- * associated with each while {@link UnassignProcedure} starts at state
- * REGION_TRANSITION_DISPATCH and state REGION_TRANSITION_QUEUE is not supported.</li>
- *
- * <li>When any step in {@link AssignProcedure} fails, failure handler
- * AssignProcedure#handleFailure(MasterProcedureEnv, RegionStateNode) re-attempts the
- * assignment by setting the procedure state to REGION_TRANSITION_QUEUE and forces
- * assignment to a different target server by setting {@link AssignProcedure#forceNewPlan}. When
- * the number of attempts reaches threshold configuration 'hbase.assignment.maximum.attempts',
- * the procedure is aborted. For {@link UnassignProcedure}, similar re-attempts are
- * intentionally not implemented. It is a 'one shot' procedure. See its class doc for how it
- * handles failure.
- * </li>
- * <li>If we find a region in an 'unexpected' state, we'll complain and retry with backoff forever.
- * The 'unexpected' state needs to be fixed either by another running Procedure or by operator
- * intervention (Regions in 'unexpected' state indicates bug or unexpected transition type).
- * For this to work, subclasses need to persist the 'attempt' counter kept in this class when
- * they do serializeStateData and restore it inside their deserializeStateData, just as they do
- * for {@link #regionInfo}.
- * </li>
- * </ul>
- * </p>
- *
- * <p>TODO: Considering it is a priority doing all we can to get make a region available as soon as
- * possible, re-attempting with any target makes sense if specified target fails in case of
- * {@link AssignProcedure}. For {@link UnassignProcedure}, our concern is preventing data loss
- * on failed unassign. See class doc for explanation.
+ * Leave here only for checking if we can successfully start the master.
+ * @deprecated Do not use any more.
+ * @see TransitRegionStateProcedure
  */
+@Deprecated
 @InterfaceAudience.Private
-public abstract class RegionTransitionProcedure
-    extends Procedure<MasterProcedureEnv>
-    implements TableProcedureInterface,
-      RemoteProcedure<MasterProcedureEnv, ServerName> {
-  private static final Logger LOG = LoggerFactory.getLogger(RegionTransitionProcedure.class);
+public abstract class RegionTransitionProcedure extends Procedure<MasterProcedureEnv>
+    implements TableProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
 
   protected final AtomicBoolean aborted = new AtomicBoolean(false);
 
   private RegionTransitionState transitionState = RegionTransitionState.REGION_TRANSITION_QUEUE;
-  /**
-   * This data member must be persisted. Expectation is that it is done by subclasses in their
-   * {@link #serializeStateData(ProcedureStateSerializer)} call, restoring {@link #regionInfo}
-   * in their {@link #deserializeStateData(ProcedureStateSerializer)} method.
-   */
+
   private RegionInfo regionInfo;
 
-  /**
-   * Like {@link #regionInfo}, the expectation is that subclasses persist the value of this
-   * data member. It is used doing backoff when Procedure gets stuck.
-   */
   private int attempt;
 
   // Required by the Procedure framework to create the procedure on replay
-  public RegionTransitionProcedure() {}
+  public RegionTransitionProcedure() {
+  }
 
   public RegionTransitionProcedure(final RegionInfo regionInfo) {
     this.regionInfo = regionInfo;
@@ -128,22 +68,10 @@ public abstract class RegionTransitionProcedure
     return regionInfo;
   }
 
-  /**
-   * This setter is for subclasses to call in their
-   * {@link #deserializeStateData(ProcedureStateSerializer)} method. Expectation is that
-   * subclasses will persist `regioninfo` in their
-   * {@link #serializeStateData(ProcedureStateSerializer)} method and then restore `regionInfo` on
-   * deserialization by calling.
-   */
   protected void setRegionInfo(final RegionInfo regionInfo) {
     this.regionInfo = regionInfo;
   }
 
-  /**
-   * This setter is for subclasses to call in their
-   * {@link #deserializeStateData(ProcedureStateSerializer)} method.
-   * @see #setRegionInfo(RegionInfo)
-   */
   protected void setAttempt(int attempt) {
     this.attempt = attempt;
   }
@@ -155,7 +83,7 @@ public abstract class RegionTransitionProcedure
   @Override
   public TableName getTableName() {
     RegionInfo hri = getRegionInfo();
-    return hri != null? hri.getTable(): null;
+    return hri != null ? hri.getTable() : null;
   }
 
   public boolean isMeta() {
@@ -168,7 +96,7 @@ public abstract class RegionTransitionProcedure
     sb.append(" table=");
     sb.append(getTableName());
     sb.append(", region=");
-    sb.append(getRegionInfo() == null? null: getRegionInfo().getEncodedName());
+    sb.append(getRegionInfo() == null ? null : getRegionInfo().getEncodedName());
   }
 
   public RegionStateNode getRegionState(final MasterProcedureEnv env) {
@@ -184,113 +112,26 @@ public abstract class RegionTransitionProcedure
   }
 
   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.
-   */
+      throws IOException, ProcedureSuspendedException;
+
   protected abstract boolean updateTransition(MasterProcedureEnv env, RegionStateNode regionNode)
-    throws IOException, ProcedureSuspendedException;
+      throws IOException, ProcedureSuspendedException;
 
   protected abstract void finishTransition(MasterProcedureEnv env, RegionStateNode regionNode)
-    throws IOException, ProcedureSuspendedException;
+      throws IOException, ProcedureSuspendedException;
 
-  protected abstract void reportTransition(MasterProcedureEnv env,
-      RegionStateNode regionNode, TransitionCode code, long seqId) throws UnexpectedStateException;
+  protected abstract void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      TransitionCode code, long seqId) throws UnexpectedStateException;
 
   @Override
   public abstract RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName serverName);
 
-  /**
-   * @return True if processing of fail is complete; the procedure will be woken from its suspend
-   * and we'll go back to running through procedure steps:
-   * otherwise if false we leave the procedure in suspended state.
-   */
-  protected abstract boolean remoteCallFailed(MasterProcedureEnv env,
-      RegionStateNode regionNode, IOException exception);
+  protected abstract boolean remoteCallFailed(MasterProcedureEnv env, RegionStateNode regionNode,
+      IOException exception);
 
   @Override
   public synchronized void remoteCallFailed(final MasterProcedureEnv env,
       final ServerName serverName, final IOException exception) {
-    final RegionStateNode regionNode = getRegionState(env);
-    LOG.warn("Remote call failed {}; {}; {}; exception={}", serverName,
-        this, regionNode.toShortString(), exception.getClass().getSimpleName(), exception);
-    if (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.
-      regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
-    }
-    // else leave the procedure in suspended state; it is waiting on another call to this callback
-  }
-
-  /**
-   * 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
-   * immediately!
-   * @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) {
-    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'.
-    getRegionState(env).getProcedureEvent().suspend();
-
-    // Tricky because the below call to addOperationToNode can fail. If it fails, we need to
-    // backtrack on stuff like the 'suspend' done above -- tricky as the 'wake' requests us -- and
-    // ditto up in the caller; it needs to undo state changes. Inside in remoteCallFailed, it does
-    // wake to undo the above suspend.
-    try {
-      env.getRemoteDispatcher().addOperationToNode(targetServer, this);
-    } catch (FailedRemoteDispatchException frde) {
-      remoteCallFailed(env, targetServer, frde);
-      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.
-    regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
-  }
-
-  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
@@ -303,108 +144,12 @@ public abstract class RegionTransitionProcedure
   }
 
   @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 (regionNode.getProcedureEvent().suspendIfNotReady(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 (regionNode.getProcedureEvent().suspendIfNotReady(this)) {
-              throw new ProcedureSuspendedException();
-            }
-            break;
-
-          case REGION_TRANSITION_FINISH:
-            // 3. wait assignment response. completion/failure
-            LOG.debug("Finishing {}; {}", this, regionNode.toShortString());
-            finishTransition(env, regionNode);
-            am.removeRegionInTransition(regionNode, this);
-            return null;
-        }
-      } while (retry);
-      // If here, success so clear out the attempt counter so we start fresh each time we get stuck.
-      this.attempt = 0;
-    } catch (IOException e) {
-      long backoff = getBackoffTime(this.attempt++);
-      LOG.warn("Failed transition, suspend {}secs {}; {}; waiting on rectified condition fixed " +
-              "by other Procedure or operator intervention", backoff / 1000, this,
-          regionNode.toShortString(), e);
-      getRegionState(env).getProcedureEvent().suspend();
-      if (getRegionState(env).getProcedureEvent().suspendIfNotReady(this)) {
-        setTimeout(Math.toIntExact(backoff));
-        setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
-        throw new ProcedureSuspendedException();
-      }
-    }
-
-    return new Procedure[] {this};
-  }
-
-  private long getBackoffTime(int attempts) {
-    long backoffTime = (long)(1000 * Math.pow(2, attempts));
-    long maxBackoffTime = 60 * 60 * 1000; // An hour. Hard-coded for for now.
-    return backoffTime < maxBackoffTime? backoffTime: maxBackoffTime;
-  }
-
-  /**
-   * At end of timeout, wake ourselves up so we run again.
-   */
-  @Override
-  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
-    setState(ProcedureProtos.ProcedureState.RUNNABLE);
-    getRegionState(env).getProcedureEvent().wake(env.getProcedureScheduler());
-    return false; // 'false' means that this procedure handled the timeout
+  protected Procedure[] execute(final MasterProcedureEnv env) {
+    return null;
   }
 
   @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 void rollback(MasterProcedureEnv env) {
   }
 
   protected abstract boolean isRollbackSupported(final RegionTransitionState state);
@@ -419,54 +164,6 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  protected boolean waitInitialized(MasterProcedureEnv env) {
-    // Unless we are assigning meta, wait for meta to be available and loaded.
-    if (isMeta()) {
-      return false;
-    }
-    AssignmentManager am = env.getAssignmentManager();
-    return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, regionInfo);
-  }
-
-  @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    // 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) {
-        // ignore, just for logging
-      }
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeRegion(this, getRegionInfo());
-  }
-
-  @Override
-  protected boolean holdLock(final MasterProcedureEnv env) {
-    return true;
-  }
-
-  @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);
-
-  @Override
   public void remoteOperationCompleted(MasterProcedureEnv env) {
     // should not be called for region operation until we modified the open/close region procedure
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
new file mode 100644
index 0000000..6925c42
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
@@ -0,0 +1,55 @@
+/**
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * Server State.
+ */
+@InterfaceAudience.Private
+enum ServerState {
+  /**
+   * Initial state. Available.
+   */
+  ONLINE,
+
+  /**
+   * Only server which carries meta can have this state. We will split wal for meta and then
+   * assign meta first before splitting other wals.
+   */
+  SPLITTING_META,
+
+  /**
+   * Indicate that the meta splitting is done. We need this state so that the UnassignProcedure
+   * for meta can safely quit. See the comments in UnassignProcedure.remoteCallFailed for more
+   * details.
+   */
+  SPLITTING_META_DONE,
+
+  /**
+   * Server expired/crashed. Currently undergoing WAL splitting.
+   */
+  SPLITTING,
+
+  /**
+   * WAL splitting done. This state will be used to tell the UnassignProcedure that it can safely
+   * quit. See the comments in UnassignProcedure.remoteCallFailed for more details.
+   */
+  OFFLINE
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
new file mode 100644
index 0000000..2042214
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
@@ -0,0 +1,128 @@
+/**
+ * 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.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * State of Server; list of hosted regions, etc.
+ */
+@InterfaceAudience.Private
+class ServerStateNode implements Comparable<ServerStateNode> {
+
+  private static final class ServerReportEvent extends ProcedureEvent<ServerName> {
+    public ServerReportEvent(final ServerName serverName) {
+      super(serverName);
+    }
+  }
+
+  private final ServerReportEvent reportEvent;
+
+  private final Set<RegionStateNode> regions;
+  private final ServerName serverName;
+
+  private volatile ServerState state = ServerState.ONLINE;
+
+  public ServerStateNode(final ServerName serverName) {
+    this.serverName = serverName;
+    this.regions = ConcurrentHashMap.newKeySet();
+    this.reportEvent = new ServerReportEvent(serverName);
+  }
+
+  public ServerName getServerName() {
+    return serverName;
+  }
+
+  public ServerState getState() {
+    return state;
+  }
+
+  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;
+  }
+
+  void setState(final ServerState state) {
+    this.state = state;
+  }
+
+  public Set<RegionStateNode> getRegions() {
+    return regions;
+  }
+
+  public int getRegionCount() {
+    return regions.size();
+  }
+
+  public ArrayList<RegionInfo> getRegionInfoList() {
+    ArrayList<RegionInfo> hris = new ArrayList<RegionInfo>(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());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 29dbabb..4e292c5 100644
--- 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
@@ -15,7 +15,6 @@
  * 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;
@@ -32,6 +31,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,13 +45,11 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-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.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -139,16 +137,6 @@ public class SplitTableRegionProcedure
   }
 
   /**
-   * Check whether there are recovered.edits in the parent closed region.
-   * @param env master env
-   * @throws IOException IOException
-   */
-  static boolean hasRecoveredEdits(MasterProcedureEnv env, RegionInfo ri) throws IOException {
-    return WALSplitter.hasRecoveredEdits(env.getMasterServices().getFileSystem(),
-        env.getMasterConfiguration(), ri);
-  }
-
-  /**
    * Check whether the region is splittable
    * @param env MasterProcedureEnv
    * @param regionToSplit parent Region to be split
@@ -169,12 +157,13 @@ public class SplitTableRegionProcedure
     if (node != null) {
       try {
         if (bestSplitRow == null || bestSplitRow.length == 0) {
-          LOG.info("splitKey isn't explicitly specified, " + " will try to find a best split key from RS");
+          LOG
+            .info("splitKey isn't explicitly specified, will try to find a best split key from RS");
         }
         // Always set bestSplitRow request as true here,
         // need to call Region#checkSplit to check it splittable or not
-        GetRegionInfoResponse response =
-            Util.getRegionInfoResponse(env, node.getRegionLocation(), node.getRegionInfo(), true);
+        GetRegionInfoResponse response = AssignmentManagerUtil.getRegionInfoResponse(env,
+          node.getRegionLocation(), node.getRegionInfo(), true);
         if(bestSplitRow == null || bestSplitRow.length == 0) {
           bestSplitRow = response.hasBestSplitRow() ? response.getBestSplitRow().toByteArray() : null;
         }
@@ -189,14 +178,17 @@ public class SplitTableRegionProcedure
     }
 
     if (!splittable) {
-      IOException e = new DoNotRetryIOException(regionToSplit.getShortNameToLog() + " NOT splittable");
-      if (splittableCheckIOE != null) e.initCause(splittableCheckIOE);
+      IOException e =
+        new DoNotRetryIOException(regionToSplit.getShortNameToLog() + " NOT splittable");
+      if (splittableCheckIOE != null) {
+        e.initCause(splittableCheckIOE);
+      }
       throw e;
     }
 
-    if(bestSplitRow == null || bestSplitRow.length == 0) {
-      throw new DoNotRetryIOException("Region not splittable because bestSplitPoint = null, "
-          + "maybe table is too small for auto split. For force split, try specifying split row");
+    if (bestSplitRow == null || bestSplitRow.length == 0) {
+      throw new DoNotRetryIOException("Region not splittable because bestSplitPoint = null, " +
+        "maybe table is too small for auto split. For force split, try specifying split row");
     }
 
     if (Bytes.equals(regionToSplit.getStartKey(), bestSplitRow)) {
@@ -205,9 +197,8 @@ public class SplitTableRegionProcedure
     }
 
     if (!regionToSplit.containsRow(bestSplitRow)) {
-      throw new DoNotRetryIOException(
-        "Split row is not inside region key range splitKey:" + Bytes.toStringBinary(splitRow) +
-        " region: " + regionToSplit);
+      throw new DoNotRetryIOException("Split row is not inside region key range splitKey:" +
+        Bytes.toStringBinary(splitRow) + " region: " + regionToSplit);
     }
   }
 
@@ -228,8 +219,19 @@ public class SplitTableRegionProcedure
     return rid;
   }
 
+  private void removeNonDefaultReplicas(MasterProcedureEnv env) throws IOException {
+    AssignmentManagerUtil.removeNonDefaultReplicas(env, Stream.of(getParentRegion()),
+      getRegionReplication(env));
+  }
+
+  private void checkClosedRegions(MasterProcedureEnv env) throws IOException {
+    // theoretically this should not happen any more after we use TRSP, but anyway let's add a check
+    // here
+    AssignmentManagerUtil.checkClosedRegion(env, getParentRegion());
+  }
+
   @Override
-  protected Flow executeFromState(final MasterProcedureEnv env, final SplitTableRegionState state)
+  protected Flow executeFromState(MasterProcedureEnv env, SplitTableRegionState state)
       throws InterruptedException {
     LOG.trace("{} execute state={}", this, state);
 
@@ -247,24 +249,15 @@ public class SplitTableRegionProcedure
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION);
           break;
         case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION:
-          addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+          addChildProcedure(createUnassignProcedures(env));
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS);
           break;
         case SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
-          if (hasRecoveredEdits(env, getRegion())) {
-            // If recovered edits, reopen parent region and then re-run the close by going back to
-            // SPLIT_TABLE_REGION_CLOSE_PARENT_REGION. We might have to cycle here a few times
-            // (TODO: Add being able to open a region in read-only mode). Open the primary replica
-            // in this case only where we just want to pickup the left-out replicated.edits.
-            LOG.info("Found recovered.edits under {}, reopen so we pickup these missed edits!",
-                getRegion().getEncodedName());
-            addChildProcedure(env.getAssignmentManager().createAssignProcedure(getParentRegion()));
-            setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION);
-          } else {
-            setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
-          }
+          checkClosedRegions(env);
+          setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS);
           break;
         case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS:
+          removeNonDefaultReplicas(env);
           createDaughterRegions(env);
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_WRITE_MAX_SEQUENCE_ID_FILE);
           break;
@@ -285,7 +278,7 @@ public class SplitTableRegionProcedure
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS);
           break;
         case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS:
-          addChildProcedure(createAssignProcedures(env, getRegionReplication(env)));
+          addChildProcedure(createAssignProcedures(env));
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_POST_OPERATION);
           break;
         case SPLIT_TABLE_REGION_POST_OPERATION:
@@ -544,24 +537,14 @@ public class SplitTableRegionProcedure
 
   /**
    * 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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
-      procs[i] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
-    }
-    env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
+  private void openParentRegion(MasterProcedureEnv env) throws IOException {
+    AssignmentManagerUtil.reopenRegionsForRollback(env, Stream.of(getParentRegion()),
+      getRegionReplication(env), getParentRegionServerName(env));
   }
 
   /**
    * Create daughter regions
-   * @param env MasterProcedureEnv
    */
   @VisibleForTesting
   public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
@@ -818,35 +801,21 @@ public class SplitTableRegionProcedure
   }
 
   private ServerName getParentRegionServerName(final MasterProcedureEnv env) {
-    return env.getMasterServices().getAssignmentManager()
-      .getRegionStates().getRegionServerOfRegion(getParentRegion());
+    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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
-      procs[i] = env.getAssignmentManager().
-          createUnassignProcedure(hri, null, true, !RegionReplicaUtil.isDefaultReplica(hri));
-    }
-    return procs;
+  private TransitRegionStateProcedure[] createUnassignProcedures(MasterProcedureEnv env)
+      throws IOException {
+    return AssignmentManagerUtil.createUnassignProceduresForSplitOrMerge(env,
+      Stream.of(getParentRegion()), getRegionReplication(env));
   }
 
-  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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_1_RI, i);
-      procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
-    }
-    for (int i = 0; i < regionReplication; ++i) {
-      final RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(daughter_2_RI, i);
-      procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
-    }
-    return procs;
+  private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env)
+      throws IOException {
+    return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env,
+      Stream.of(daughter_1_RI, daughter_2_RI), getRegionReplication(env),
+      getParentRegionServerName(env));
   }
 
   private int getRegionReplication(final MasterProcedureEnv env) throws IOException {


[6/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 70a9680..9b020c8 100644
--- 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
@@ -19,14 +19,12 @@ package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -34,22 +32,23 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.RegionException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.YouAreDeadException;
+import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-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.exceptions.UnexpectedStateException;
 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;
@@ -59,14 +58,10 @@ 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;
 import org.apache.hadoop.hbase.master.balancer.FavoredStochasticBalancer;
 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.ServerCrashException;
 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
@@ -90,7 +85,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 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.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
@@ -149,10 +143,6 @@ public class AssignmentManager implements ServerListener {
   private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
-  /** Listeners that are called on assignment events. */
-  private final CopyOnWriteArrayList<AssignmentListener> listeners =
-      new CopyOnWriteArrayList<AssignmentListener>();
-
   private final MetricsAssignmentManager metrics;
   private final RegionInTransitionChore ritChore;
   private final MasterServices master;
@@ -216,16 +206,55 @@ public class AssignmentManager implements ServerListener {
     // it could be null in some tests
     if (zkw != null) {
       RegionState regionState = MetaTableLocator.getMetaRegionState(zkw);
-      RegionStateNode regionStateNode =
+      RegionStateNode regionNode =
         regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
-      synchronized (regionStateNode) {
-        regionStateNode.setRegionLocation(regionState.getServerName());
-        regionStateNode.setState(regionState.getState());
+      regionNode.lock();
+      try {
+        regionNode.setRegionLocation(regionState.getServerName());
+        regionNode.setState(regionState.getState());
         setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
+      } finally {
+        regionNode.unlock();
       }
     }
   }
 
+  /**
+   * Create RegionStateNode based on the TRSP list, and attach the TRSP to the RegionStateNode.
+   * <p>
+   * This is used to restore the RIT region list, so we do not need to restore it in the loadingMeta
+   * method below. And it is also very important as now before submitting a TRSP, we need to attach
+   * it to the RegionStateNode, which acts like a guard, so we need to restore this information at
+   * the very beginning, before we start processing any procedures.
+   */
+  public void setupRIT(List<TransitRegionStateProcedure> procs) {
+    procs.forEach(proc -> {
+      RegionInfo regionInfo = proc.getRegion();
+      RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
+      TransitRegionStateProcedure existingProc = regionNode.getProcedure();
+      if (existingProc != null) {
+        // This is possible, as we will detach the procedure from the RSN before we
+        // actually finish the procedure. This is because that, we will update the region state
+        // directly in the reportTransition method for TRSP, and theoretically the region transition
+        // has been done, so we need to detach the procedure from the RSN. But actually the
+        // procedure has not been marked as done in the pv2 framework yet, so it is possible that we
+        // schedule a new TRSP immediately and when arriving here, we will find out that there are
+        // multiple TRSPs for the region. But we can make sure that, only the last one can take the
+        // charge, the previous ones should have all been finished already.
+        // So here we will compare the proc id, the greater one will win.
+        if (existingProc.getProcId() < proc.getProcId()) {
+          // the new one wins, unset and set it to the new one below
+          regionNode.unsetProcedure(existingProc);
+        } else {
+          // the old one wins, skip
+          return;
+        }
+      }
+      LOG.info("Attach {} to {} to restore RIT", proc, regionNode);
+      regionNode.setProcedure(proc);
+    });
+  }
+
   public void stop() {
     if (!running.compareAndSet(true, false)) {
       return;
@@ -288,22 +317,6 @@ public class AssignmentManager implements ServerListener {
     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 RegionStates getRegionStates() {
     return regionStates;
   }
@@ -513,51 +526,89 @@ public class AssignmentManager implements ServerListener {
   private List<RegionInfo> getSystemTables(ServerName serverName) {
     Set<RegionStateNode> regions = this.getRegionStates().getServerNode(serverName).getRegions();
     if (regions == null) {
-      return new ArrayList<>();
+      return Collections.emptyList();
     }
-    return regions.stream()
-        .map(RegionStateNode::getRegionInfo)
-        .filter(r -> r.getTable().isSystemTable())
-        .collect(Collectors.toList());
+    return regions.stream().map(RegionStateNode::getRegionInfo)
+      .filter(r -> r.getTable().isSystemTable()).collect(Collectors.toList());
   }
 
-  public void assign(final RegionInfo regionInfo, ServerName sn) throws IOException {
-    AssignProcedure proc = createAssignProcedure(regionInfo, sn);
-    ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
+  private void preTransitCheck(RegionStateNode regionNode, RegionState.State[] expectedStates)
+      throws HBaseIOException {
+    if (regionNode.getProcedure() != null) {
+      throw new HBaseIOException(regionNode + " is currently in transition");
+    }
+    if (!regionNode.isInState(expectedStates)) {
+      throw new DoNotRetryRegionException("Unexpected state for " + regionNode);
+    }
+    if (getTableStateManager().isTableState(regionNode.getTable(), TableState.State.DISABLING,
+      TableState.State.DISABLED)) {
+      throw new DoNotRetryIOException(regionNode.getTable() + " is disabled for " + regionNode);
+    }
   }
 
-  public void assign(final RegionInfo regionInfo) throws IOException {
-    AssignProcedure proc = createAssignProcedure(regionInfo);
+  public void assign(RegionInfo regionInfo, ServerName sn) throws IOException {
+    // TODO: should we use getRegionStateNode?
+    RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
+    TransitRegionStateProcedure proc;
+    regionNode.lock();
+    try {
+      preTransitCheck(regionNode, RegionStates.STATES_EXPECTED_ON_OPEN);
+      proc = TransitRegionStateProcedure.assign(getProcedureEnvironment(), regionInfo, sn);
+      regionNode.setProcedure(proc);
+    } finally {
+      regionNode.unlock();
+    }
     ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
   }
 
-  public void unassign(final RegionInfo regionInfo) throws IOException {
-    unassign(regionInfo, false);
+  public void assign(RegionInfo regionInfo) throws IOException {
+    assign(regionInfo, null);
   }
 
-  public void unassign(final RegionInfo regionInfo, final boolean forceNewPlan)
-  throws IOException {
-    // TODO: rename this reassign
-    RegionStateNode node = this.regionStates.getRegionStateNode(regionInfo);
-    ServerName destinationServer = node.getRegionLocation();
-    if (destinationServer == null) {
-      throw new UnexpectedStateException("DestinationServer is null; Assigned? " + node.toString());
+  public void unassign(RegionInfo regionInfo) throws IOException {
+    RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
+    if (regionNode == null) {
+      throw new UnknownRegionException("No RegionState found for " + regionInfo.getEncodedName());
+    }
+    TransitRegionStateProcedure proc;
+    regionNode.lock();
+    try {
+      preTransitCheck(regionNode, RegionStates.STATES_EXPECTED_ON_CLOSE);
+      proc = TransitRegionStateProcedure.unassign(getProcedureEnvironment(), regionInfo);
+      regionNode.setProcedure(proc);
+    } finally {
+      regionNode.unlock();
     }
-    assert destinationServer != null; node.toString();
-    UnassignProcedure proc = createUnassignProcedure(regionInfo, destinationServer, forceNewPlan);
     ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
   }
 
-  public void move(final RegionInfo regionInfo) throws IOException {
-    RegionStateNode node = this.regionStates.getRegionStateNode(regionInfo);
-    ServerName sourceServer = node.getRegionLocation();
-    RegionPlan plan = new RegionPlan(regionInfo, sourceServer, null);
-    MoveRegionProcedure proc = createMoveRegionProcedure(plan);
+  private TransitRegionStateProcedure createMoveRegionProcedure(RegionInfo regionInfo,
+      ServerName targetServer) throws HBaseIOException {
+    RegionStateNode regionNode = this.regionStates.getRegionStateNode(regionInfo);
+    if (regionNode == null) {
+      throw new UnknownRegionException("No RegionState found for " + regionInfo.getEncodedName());
+    }
+    TransitRegionStateProcedure proc;
+    regionNode.lock();
+    try {
+      preTransitCheck(regionNode, RegionStates.STATES_EXPECTED_ON_CLOSE);
+      regionNode.checkOnline();
+      proc = TransitRegionStateProcedure.move(getProcedureEnvironment(), regionInfo, targetServer);
+      regionNode.setProcedure(proc);
+    } finally {
+      regionNode.unlock();
+    }
+    return proc;
+  }
+
+  public void move(RegionInfo regionInfo) throws IOException {
+    TransitRegionStateProcedure proc = createMoveRegionProcedure(regionInfo, null);
     ProcedureSyncWait.submitAndWaitProcedure(master.getMasterProcedureExecutor(), proc);
   }
 
-  public Future<byte[]> moveAsync(final RegionPlan regionPlan) throws HBaseIOException {
-    MoveRegionProcedure proc = createMoveRegionProcedure(regionPlan);
+  public Future<byte[]> moveAsync(RegionPlan regionPlan) throws HBaseIOException {
+    TransitRegionStateProcedure proc =
+      createMoveRegionProcedure(regionPlan.getRegionInfo(), regionPlan.getDestination());
     return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
   }
 
@@ -569,7 +620,7 @@ public class AssignmentManager implements ServerListener {
   @VisibleForTesting
   // TODO: Remove this?
   public boolean waitForAssignment(final RegionInfo regionInfo, final long timeout)
-  throws IOException {
+      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.
@@ -577,24 +628,27 @@ public class AssignmentManager implements ServerListener {
     // Something badly wrong if takes ten seconds to register a region.
     long endTime = startTime + 10000;
     while ((node = regionStates.getRegionStateNode(regionInfo)) == null && isRunning() &&
-        System.currentTimeMillis() < endTime) {
+      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.");
+      if (!isRunning()) {
+        return false;
+      }
+      throw new RegionException(
+        regionInfo.getRegionNameAsString() + " never registered with Assigment.");
     }
 
-    RegionTransitionProcedure proc = node.getProcedure();
+    TransitRegionStateProcedure proc = node.getProcedure();
     if (proc == null) {
       throw new NoSuchProcedureException(node.toString());
     }
 
-    ProcedureSyncWait.waitForProcedureToCompleteIOE(
-      master.getMasterProcedureExecutor(), proc, timeout);
+    ProcedureSyncWait.waitForProcedureToCompleteIOE(master.getMasterProcedureExecutor(), proc,
+      timeout);
     return true;
   }
 
@@ -604,22 +658,23 @@ public class AssignmentManager implements ServerListener {
 
   /**
    * Create round-robin assigns. Use on table creation to distribute out regions across cluster.
-   * @return AssignProcedures made out of the passed in <code>hris</code> and a call
-   * to the balancer to populate the assigns with targets chosen using round-robin (default
-   * balancer scheme). If at assign-time, the target chosen is no longer up, thats fine,
-   * the AssignProcedure will ask the balancer for a new target, and so on.
+   * @return AssignProcedures made out of the passed in <code>hris</code> and a call to the balancer
+   *         to populate the assigns with targets chosen using round-robin (default balancer
+   *         scheme). If at assign-time, the target chosen is no longer up, thats fine, the
+   *         AssignProcedure will ask the balancer for a new target, and so on.
    */
-  public AssignProcedure[] createRoundRobinAssignProcedures(final List<RegionInfo> hris) {
+  public TransitRegionStateProcedure[] createRoundRobinAssignProcedures(
+      List<RegionInfo> hris) {
     if (hris.isEmpty()) {
-      return null;
+      return new TransitRegionStateProcedure[0];
     }
     try {
       // Ask the balancer to assign our regions. Pass the regions en masse. The balancer can do
       // a better job if it has all the assignments in the one lump.
       Map<ServerName, List<RegionInfo>> assignments = getBalancer().roundRobinAssignment(hris,
-          this.master.getServerManager().createDestinationServersList(null));
+        this.master.getServerManager().createDestinationServersList(null));
       // Return mid-method!
-      return createAssignProcedures(assignments, hris.size());
+      return createAssignProcedures(assignments);
     } catch (HBaseIOException hioe) {
       LOG.warn("Failed roundRobinAssignment", hioe);
     }
@@ -627,130 +682,97 @@ public class AssignmentManager implements ServerListener {
     return createAssignProcedures(hris);
   }
 
-  /**
-   * Create an array of AssignProcedures w/o specifying a target server.
-   * If no target server, at assign time, we will try to use the former location of the region
-   * if one exists. This is how we 'retain' the old location across a server restart.
-   * Used by {@link ServerCrashProcedure} assigning regions on a server that has crashed (SCP is
-   * also used across a cluster-restart just-in-case to ensure we do cleanup of any old WALs or
-   * server processes).
-   */
-  public AssignProcedure[] createAssignProcedures(final List<RegionInfo> hris) {
-    if (hris.isEmpty()) {
-      return null;
-    }
-    int index = 0;
-    AssignProcedure [] procedures = new AssignProcedure[hris.size()];
-    for (RegionInfo hri : hris) {
-      // Sort the procedures so meta and system regions are first in the returned array.
-      procedures[index++] = createAssignProcedure(hri);
-    }
-    if (procedures.length > 1) {
-      // Sort the procedures so meta and system regions are first in the returned array.
-      Arrays.sort(procedures, AssignProcedure.COMPARATOR);
-    }
-    return procedures;
-  }
-
-  // Make this static for the method below where we use it typing the AssignProcedure array we
-  // return as result.
-  private static final AssignProcedure [] ASSIGN_PROCEDURE_ARRAY_TYPE = new AssignProcedure[] {};
-
-  /**
-   * @param assignments Map of assignments from which we produce an array of AssignProcedures.
-   * @param size Count of assignments to make (the caller may know the total count)
-   * @return Assignments made from the passed in <code>assignments</code>
-   */
-  private AssignProcedure[] createAssignProcedures(Map<ServerName, List<RegionInfo>> assignments,
-      int size) {
-    List<AssignProcedure> procedures = new ArrayList<>(size > 0? size: 8/*Arbitrary*/);
-    for (Map.Entry<ServerName, List<RegionInfo>> e: assignments.entrySet()) {
-      for (RegionInfo ri: e.getValue()) {
-        AssignProcedure ap = createAssignProcedure(ri, e.getKey());
-        ap.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
-        procedures.add(ap);
+  @VisibleForTesting
+  static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
+    if (left.getRegion().isMetaRegion()) {
+      if (right.getRegion().isMetaRegion()) {
+        return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
       }
+      return -1;
+    } else if (right.getRegion().isMetaRegion()) {
+      return +1;
     }
-    if (procedures.size() > 1) {
-      // Sort the procedures so meta and system regions are first in the returned array.
-      procedures.sort(AssignProcedure.COMPARATOR);
+    if (left.getRegion().getTable().isSystemTable()) {
+      if (right.getRegion().getTable().isSystemTable()) {
+        return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
+      }
+      return -1;
+    } else if (right.getRegion().getTable().isSystemTable()) {
+      return +1;
     }
-    return procedures.toArray(ASSIGN_PROCEDURE_ARRAY_TYPE);
+    return RegionInfo.COMPARATOR.compare(left.getRegion(), right.getRegion());
   }
 
-  // Needed for the following method so it can type the created Array we retur n
-  private static final UnassignProcedure [] UNASSIGN_PROCEDURE_ARRAY_TYPE =
-      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));
+  private TransitRegionStateProcedure createAssignProcedure(RegionStateNode regionNode,
+      ServerName targetServer) {
+    TransitRegionStateProcedure proc;
+    regionNode.lock();
+    try {
+      assert regionNode.getProcedure() == null;
+      proc = TransitRegionStateProcedure.assign(getProcedureEnvironment(),
+        regionNode.getRegionInfo(), targetServer);
+      regionNode.setProcedure(proc);
+    } finally {
+      regionNode.unlock();
     }
-    return procs.toArray(UNASSIGN_PROCEDURE_ARRAY_TYPE);
+    return proc;
   }
 
   /**
-   * Called by things like DisableTableProcedure to get a list of UnassignProcedure
-   * to unassign the regions of the table.
+   * Create an array of TransitRegionStateProcedure w/o specifying a target server.
+   * <p/>
+   * If no target server, at assign time, we will try to use the former location of the region if
+   * one exists. This is how we 'retain' the old location across a server restart.
+   * <p/>
+   * Should only be called when you can make sure that no one can touch these regions other than
+   * you. For example, when you are creating table.
    */
-  public UnassignProcedure[] createUnassignProcedures(final TableName tableName) {
-    return createUnassignProcedures(regionStates.getTableRegionStateNodes(tableName));
+  public TransitRegionStateProcedure[] createAssignProcedures(List<RegionInfo> hris) {
+    return hris.stream().map(hri -> regionStates.getOrCreateRegionStateNode(hri))
+      .map(regionNode -> createAssignProcedure(regionNode, null)).sorted(AssignmentManager::compare)
+      .toArray(TransitRegionStateProcedure[]::new);
   }
 
-  public AssignProcedure createAssignProcedure(final RegionInfo regionInfo) {
-    AssignProcedure proc = new AssignProcedure(regionInfo);
-    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
-    return proc;
-  }
-
-  public AssignProcedure createAssignProcedure(final RegionInfo regionInfo,
-      final ServerName targetServer) {
-    AssignProcedure proc = new AssignProcedure(regionInfo, targetServer);
-    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
-    return proc;
-  }
-
-  UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
-      final ServerName destinationServer, final boolean force) {
-    return createUnassignProcedure(regionInfo, destinationServer, force, false);
-  }
-
-  UnassignProcedure createUnassignProcedure(final RegionInfo regionInfo,
-      final ServerName destinationServer, final boolean force,
-      final boolean removeAfterUnassigning) {
-    // 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, removeAfterUnassigning);
-    proc.setOwner(getProcedureEnvironment().getRequestUser().getShortName());
-    return proc;
+  /**
+   * @param assignments Map of assignments from which we produce an array of AssignProcedures.
+   * @return Assignments made from the passed in <code>assignments</code>
+   */
+  private TransitRegionStateProcedure[] createAssignProcedures(
+      Map<ServerName, List<RegionInfo>> assignments) {
+    return assignments.entrySet().stream()
+      .flatMap(e -> e.getValue().stream().map(hri -> regionStates.getOrCreateRegionStateNode(hri))
+        .map(regionNode -> createAssignProcedure(regionNode, e.getKey())))
+      .sorted(AssignmentManager::compare).toArray(TransitRegionStateProcedure[]::new);
   }
 
-  private MoveRegionProcedure createMoveRegionProcedure(RegionPlan plan) throws HBaseIOException {
-    if (plan.getRegionInfo().getTable().isSystemTable()) {
-      List<ServerName> exclude = getExcludedServersForSystemTable();
-      if (plan.getDestination() != null && exclude.contains(plan.getDestination())) {
-        try {
-          LOG.info("Can not move " + plan.getRegionInfo() + " to " + plan.getDestination() +
-            " because the server is not with highest version");
-          plan.setDestination(getBalancer().randomAssignment(plan.getRegionInfo(),
-            this.master.getServerManager().createDestinationServersList(exclude)));
-        } catch (HBaseIOException e) {
-          LOG.warn(e.toString(), e);
+  /**
+   * Called by DisableTableProcedure to unassign all the regions for a table.
+   */
+  public TransitRegionStateProcedure[] createUnassignProceduresForDisabling(TableName tableName) {
+    return regionStates.getTableRegionStateNodes(tableName).stream().map(regionNode -> {
+      regionNode.lock();
+      try {
+        if (!regionStates.include(regionNode, false) ||
+          regionStates.isRegionOffline(regionNode.getRegionInfo())) {
+          return null;
+        }
+        // As in DisableTableProcedure, we will hold the xlock for table, so we can make sure that
+        // this procedure has not been executed yet, as TRSP will hold the shared lock for table all
+        // the time. So here we will unset it and when it is actually executed, it will find that
+        // the attach procedure is not itself and quit immediately.
+        if (regionNode.getProcedure() != null) {
+          regionNode.unsetProcedure(regionNode.getProcedure());
         }
+        TransitRegionStateProcedure proc = TransitRegionStateProcedure
+          .unassign(getProcedureEnvironment(), regionNode.getRegionInfo());
+        regionNode.setProcedure(proc);
+        return proc;
+      } finally {
+        regionNode.unlock();
       }
-    }
-    return new MoveRegionProcedure(getProcedureEnvironment(), plan, true);
+    }).filter(p -> p != null).toArray(TransitRegionStateProcedure[]::new);
   }
 
-
   public SplitTableRegionProcedure createSplitProcedure(final RegionInfo regionToSplit,
       final byte[] splitKey) throws IOException {
     return new SplitTableRegionProcedure(getProcedureEnvironment(), regionToSplit, splitKey);
@@ -780,8 +802,7 @@ public class AssignmentManager implements ServerListener {
   // ============================================================================================
   // TODO: Move this code in MasterRpcServices and call on specific event?
   public ReportRegionStateTransitionResponse reportRegionStateTransition(
-      final ReportRegionStateTransitionRequest req)
-  throws PleaseHoldException {
+      final ReportRegionStateTransitionRequest req) throws PleaseHoldException {
     final ReportRegionStateTransitionResponse.Builder builder =
         ReportRegionStateTransitionResponse.newBuilder();
     final ServerName serverName = ProtobufUtil.toServerName(req.getServer());
@@ -819,7 +840,7 @@ public class AssignmentManager implements ServerListener {
         }
       }
     } catch (PleaseHoldException e) {
-      if (LOG.isTraceEnabled()) LOG.trace("Failed transition " + e.getMessage());
+      LOG.trace("Failed transition ", e);
       throw e;
     } catch (UnsupportedOperationException|IOException e) {
       // TODO: at the moment we have a single error message and the RS will abort
@@ -830,56 +851,53 @@ public class AssignmentManager implements ServerListener {
     return builder.build();
   }
 
-  private void updateRegionTransition(final ServerName serverName, final TransitionCode state,
-      final RegionInfo regionInfo, final long seqId)
-      throws PleaseHoldException, UnexpectedStateException {
+  private void updateRegionTransition(ServerName serverName, TransitionCode state,
+      RegionInfo regionInfo, long seqId) throws IOException {
     checkMetaLoaded(regionInfo);
 
-    final RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
+    RegionStateNode regionNode = regionStates.getRegionStateNode(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));
     }
+    LOG.trace("Update region transition serverName={} region={} regionState={}", serverName,
+      regionNode, state);
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Update region transition serverName=%s region=%s regionState=%s",
-        serverName, regionNode, state));
-    }
-
-    final ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
-    if (!reportTransition(regionNode, serverNode, state, seqId)) {
-      // Don't log WARN if shutting down cluster; during shutdown. Avoid the below messages:
-      // 2018-08-13 10:45:10,551 WARN ...AssignmentManager: No matching procedure found for
-      //   rit=OPEN, location=ve0538.halxg.cloudera.com,16020,1533493000958,
-      //   table=IntegrationTestBigLinkedList, region=65ab289e2fc1530df65f6c3d7cde7aa5 transition
-      //   to CLOSED
-      // These happen because on cluster shutdown, we currently let the RegionServers close
-      // regions. This is the only time that region close is not run by the Master (so cluster
-      // goes down fast). Consider changing it so Master runs all shutdowns.
-      if (this.master.getServerManager().isClusterShutdown() &&
+    ServerStateNode serverNode = regionStates.getOrCreateServer(serverName);
+    regionNode.lock();
+    try {
+      if (!reportTransition(regionNode, serverNode, state, seqId)) {
+        // Don't log WARN if shutting down cluster; during shutdown. Avoid the below messages:
+        // 2018-08-13 10:45:10,551 WARN ...AssignmentManager: No matching procedure found for
+        // rit=OPEN, location=ve0538.halxg.cloudera.com,16020,1533493000958,
+        // table=IntegrationTestBigLinkedList, region=65ab289e2fc1530df65f6c3d7cde7aa5 transition
+        // to CLOSED
+        // These happen because on cluster shutdown, we currently let the RegionServers close
+        // regions. This is the only time that region close is not run by the Master (so cluster
+        // goes down fast). Consider changing it so Master runs all shutdowns.
+        if (this.master.getServerManager().isClusterShutdown() &&
           state.equals(TransitionCode.CLOSED)) {
-        LOG.info("RegionServer {} {}", state, regionNode.getRegionInfo().getEncodedName());
-      } else {
-        LOG.warn("No matching procedure found for {} transition to {}", regionNode, state);
+          LOG.info("RegionServer {} {}", state, regionNode.getRegionInfo().getEncodedName());
+        } else {
+          LOG.warn("No matching procedure found for {} transition to {}", regionNode, state);
+        }
       }
+    } finally {
+      regionNode.unlock();
     }
   }
 
-  // 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);
+  private boolean reportTransition(RegionStateNode regionNode, ServerStateNode serverNode,
+      TransitionCode state, long seqId) throws IOException {
+    ServerName serverName = serverNode.getServerName();
+    TransitRegionStateProcedure proc = regionNode.getProcedure();
+    if (proc == null) {
+      return false;
     }
+    proc.reportTransition(master.getMasterProcedureExecutor().getEnvironment(), regionNode,
+      serverName, state, seqId);
     return true;
   }
 
@@ -984,10 +1002,9 @@ public class AssignmentManager implements ServerListener {
     wakeServerReportEvent(serverNode);
   }
 
-  void checkOnlineRegionsReportForMeta(final ServerStateNode serverNode,
-      final Set<byte[]> regionNames) {
+  void checkOnlineRegionsReportForMeta(ServerStateNode serverNode, Set<byte[]> regionNames) {
     try {
-      for (byte[] regionName: regionNames) {
+      for (byte[] regionName : regionNames) {
         final RegionInfo hri = getMetaRegionFromName(regionName);
         if (hri == null) {
           if (LOG.isTraceEnabled()) {
@@ -997,18 +1014,23 @@ public class AssignmentManager implements ServerListener {
           continue;
         }
 
-        final RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(hri);
+        RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(hri);
         LOG.info("META REPORTED: " + regionNode);
-        if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
-          LOG.warn("META REPORTED but no procedure found (complete?); set location=" +
+        regionNode.lock();
+        try {
+          if (!reportTransition(regionNode, serverNode, TransitionCode.OPENED, 0)) {
+            LOG.warn("META REPORTED but no procedure found (complete?); set location=" +
               serverNode.getServerName());
-          regionNode.setRegionLocation(serverNode.getServerName());
-        } else if (LOG.isTraceEnabled()) {
-          LOG.trace("META REPORTED: " + regionNode);
+            regionNode.setRegionLocation(serverNode.getServerName());
+          } else if (LOG.isTraceEnabled()) {
+            LOG.trace("META REPORTED: " + regionNode);
+          }
+        } finally {
+          regionNode.unlock();
         }
       }
-    } catch (UnexpectedStateException e) {
-      final ServerName serverName = serverNode.getServerName();
+    } catch (IOException e) {
+      ServerName serverName = serverNode.getServerName();
       LOG.warn("KILLING " + serverName + ": " + e.getMessage());
       killRegionServer(serverNode);
     }
@@ -1019,12 +1041,15 @@ public class AssignmentManager implements ServerListener {
     final ServerName serverName = serverNode.getServerName();
     try {
       for (byte[] regionName: regionNames) {
-        if (!isRunning()) return;
+        if (!isRunning()) {
+          return;
+        }
         final RegionStateNode regionNode = regionStates.getRegionStateNodeFromName(regionName);
         if (regionNode == null) {
           throw new UnexpectedStateException("Not online: " + Bytes.toStringBinary(regionName));
         }
-        synchronized (regionNode) {
+        regionNode.lock();
+        try {
           if (regionNode.isInState(State.OPENING, State.OPEN)) {
             if (!regionNode.getRegionLocation().equals(serverName)) {
               throw new UnexpectedStateException(regionNode.toString() +
@@ -1050,9 +1075,11 @@ public class AssignmentManager implements ServerListener {
                 " reported an unexpected OPEN; time since last update=" + diff);
             }
           }
+        } finally {
+          regionNode.unlock();
         }
       }
-    } catch (UnexpectedStateException e) {
+    } catch (IOException e) {
       LOG.warn("Killing " + serverName + ": " + e.getMessage());
       killRegionServer(serverNode);
       throw (YouAreDeadException)new YouAreDeadException(e.getMessage()).initCause(e);
@@ -1267,29 +1294,20 @@ public class AssignmentManager implements ServerListener {
 
           localState = State.OFFLINE;
         }
-        final RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
-        synchronized (regionNode) {
-          if (!regionNode.isInTransition()) {
-            regionNode.setState(localState);
-            regionNode.setLastHost(lastHost);
-            regionNode.setRegionLocation(regionLocation);
-            regionNode.setOpenSeqNum(openSeqNum);
-
-            if (localState == State.OPEN) {
-              assert regionLocation != null : "found null region location for " + regionNode;
-              regionStates.addRegionToServer(regionNode);
-            } else if (localState == State.OFFLINE || regionInfo.isOffline()) {
-              regionStates.addToOfflineRegions(regionNode);
-            } else if (localState == State.CLOSED && getTableStateManager().
-                isTableState(regionNode.getTable(), TableState.State.DISABLED,
-                TableState.State.DISABLING)) {
-              // The region is CLOSED and the table is DISABLED/ DISABLING, there is nothing to
-              // schedule; the region is inert.
-            } else {
-              // These regions should have a procedure in replay
-              regionStates.addRegionInTransition(regionNode, null);
-            }
-          }
+        RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
+        // Do not need to lock on regionNode, as we can make sure that before we finish loading
+        // meta, all the related procedures can not be executed. The only exception is formeta
+        // region related operations, but here we do not load the informations for meta region.
+        regionNode.setState(localState);
+        regionNode.setLastHost(lastHost);
+        regionNode.setRegionLocation(regionLocation);
+        regionNode.setOpenSeqNum(openSeqNum);
+
+        if (localState == State.OPEN) {
+          assert regionLocation != null : "found null region location for " + regionNode;
+          regionStates.addRegionToServer(regionNode);
+        } else if (localState == State.OFFLINE || regionInfo.isOffline()) {
+          regionStates.addToOfflineRegions(regionNode);
         }
       }
     });
@@ -1335,9 +1353,11 @@ public class AssignmentManager implements ServerListener {
   }
 
   public void offlineRegion(final RegionInfo regionInfo) {
-    // TODO used by MasterRpcServices ServerCrashProcedure
-    final RegionStateNode node = regionStates.getRegionStateNode(regionInfo);
-    if (node != null) node.offline();
+    // TODO used by MasterRpcServices
+    RegionStateNode node = regionStates.getRegionStateNode(regionInfo);
+    if (node != null) {
+      node.offline();
+    }
   }
 
   public void onlineRegion(final RegionInfo regionInfo, final ServerName serverName) {
@@ -1373,16 +1393,6 @@ public class AssignmentManager implements ServerListener {
   // ============================================================================================
   //  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();
   }
@@ -1401,102 +1411,82 @@ public class AssignmentManager implements ServerListener {
   }
 
   // ============================================================================================
-  //  TODO: Region Status update
+  //  Region Status update
+  //  Should only be called in TransitRegionStateProcedure
   // ============================================================================================
-  private void sendRegionOpenedNotification(final RegionInfo 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 RegionInfo 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) {
-      regionNode.transitionState(State.OPENING, RegionStates.STATES_EXPECTED_ON_OPEN);
-      regionStates.addRegionToServer(regionNode);
-      regionStateStore.updateRegionLocation(regionNode);
-    }
+  // should be called within the synchronized block of RegionStateNode
+  void regionOpening(RegionStateNode regionNode) throws IOException {
+    regionNode.transitionState(State.OPENING, RegionStates.STATES_EXPECTED_ON_OPEN);
+    regionStateStore.updateRegionLocation(regionNode);
 
+    regionStates.addRegionToServer(regionNode);
     // 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?
+  // should be called within the synchronized block of RegionStateNode.
+  // The parameter 'giveUp' means whether we will try to open the region again, if it is true, then
+  // we will persist the FAILED_OPEN state into hbase:meta.
+  void regionFailedOpen(RegionStateNode regionNode, boolean giveUp) throws IOException {
+    if (regionNode.getRegionLocation() != null) {
+      regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
     }
-    if (opening) {
-      // TODO: Metrics. Do opposite of metrics.incrementOperationCounter();
+    if (giveUp) {
+      regionNode.setState(State.FAILED_OPEN);
+      regionNode.setRegionLocation(null);
+      regionStateStore.updateRegionLocation(regionNode);
     }
   }
 
-  public void markRegionAsOpened(final RegionStateNode regionNode) throws IOException {
-    final RegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
-      if (isMetaRegion(hri)) {
-        // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
-        // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
-        // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
-        // on table that contains state.
-        setMetaAssigned(hri, true);
-      }
-      regionStates.addRegionToServer(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);
-      sendRegionOpenedNotification(hri, regionNode.getRegionLocation());
+  // should be called within the synchronized block of RegionStateNode
+  void regionOpened(RegionStateNode regionNode) throws IOException {
+    regionNode.transitionState(State.OPEN, RegionStates.STATES_EXPECTED_ON_OPEN);
+    // 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);
+
+    RegionInfo hri = regionNode.getRegionInfo();
+    if (isMetaRegion(hri)) {
+      // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
+      // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
+      // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
+      // on table that contains state.
+      setMetaAssigned(hri, true);
     }
+    regionStates.addRegionToServer(regionNode);
+    regionStates.removeFromFailedOpen(hri);
   }
 
-  public void markRegionAsClosing(final RegionStateNode regionNode) throws IOException {
-    final RegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      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)) {
-        setMetaAssigned(hri, false);
-      }
-      regionStates.addRegionToServer(regionNode);
-      regionStateStore.updateRegionLocation(regionNode);
-    }
+  // should be called within the synchronized block of RegionStateNode
+  void regionClosing(RegionStateNode regionNode) throws IOException {
+    regionNode.transitionState(State.CLOSING, RegionStates.STATES_EXPECTED_ON_CLOSE);
+    regionStateStore.updateRegionLocation(regionNode);
 
+    RegionInfo hri = regionNode.getRegionInfo();
+    // Set meta has not initialized early. so people trying to create/edit tables will wait
+    if (isMetaRegion(hri)) {
+      setMetaAssigned(hri, false);
+    }
+    regionStates.addRegionToServer(regionNode);
     // update the operation count metrics
     metrics.incrementOperationCounter();
   }
 
-  public void undoRegionAsClosing(final RegionStateNode regionNode) {
-    // TODO: Metrics. Do opposite of metrics.incrementOperationCounter();
-    // There is nothing to undo?
-  }
-
-  public void markRegionAsClosed(final RegionStateNode regionNode) throws IOException {
-    final RegionInfo hri = regionNode.getRegionInfo();
-    synchronized (regionNode) {
-      regionNode.transitionState(State.CLOSED, RegionStates.STATES_EXPECTED_ON_CLOSE);
-      regionStates.removeRegionFromServer(regionNode.getRegionLocation(), regionNode);
-      regionNode.setLastHost(regionNode.getRegionLocation());
+  // should be called within the synchronized block of RegionStateNode
+  // The parameter 'normally' means whether we are closed cleanly, if it is true, then it means that
+  // we are closed due to a RS crash.
+  void regionClosed(RegionStateNode regionNode, boolean normally) throws IOException {
+    regionNode.transitionState(normally ? State.CLOSED : State.ABNORMALLY_CLOSED,
+      RegionStates.STATES_EXPECTED_ON_CLOSE);
+    ServerName loc = regionNode.getRegionLocation();
+    if (loc != null) {
+      // could be a retry so add a check here to avoid set the lastHost to null.
+      regionNode.setLastHost(loc);
       regionNode.setRegionLocation(null);
-      regionStateStore.updateRegionLocation(regionNode);
-      sendRegionClosedNotification(hri);
+      regionStates.removeRegionFromServer(loc, regionNode);
     }
+    regionStateStore.updateRegionLocation(regionNode);
   }
 
   public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
@@ -1829,76 +1819,4 @@ public class AssignmentManager implements ServerListener {
   private void killRegionServer(final ServerStateNode serverNode) {
     master.getServerManager().expireServer(serverNode.getServerName());
   }
-
-  /**
-   * <p>
-   * This is a very particular check. The {@link org.apache.hadoop.hbase.master.ServerManager} is
-   * where you go to check on state of 'Servers', what Servers are online, etc.
-   * </p>
-   * <p>
-   * Here we are checking the state of a server that is post expiration, a ServerManager function
-   * that moves a server from online to dead. Here we are seeing if the server has moved beyond a
-   * particular point in the recovery process such that it is safe to move on with assigns; etc.
-   * </p>
-   * <p>
-   * For now it is only used in
-   * {@link UnassignProcedure#remoteCallFailed(MasterProcedureEnv, RegionStateNode, IOException)} to
-   * see whether we can safely quit without losing data.
-   * </p>
-   * @param meta whether to check for meta log splitting
-   * @return {@code true} if the server does not exist or the log splitting is done, i.e, the server
-   *         is in OFFLINE state, or for meta log, is in SPLITTING_META_DONE state. If null,
-   *         presumes the ServerStateNode was cleaned up by SCP.
-   * @see UnassignProcedure#remoteCallFailed(MasterProcedureEnv, RegionStateNode, IOException)
-   */
-  boolean isLogSplittingDone(ServerName serverName, boolean meta) {
-    ServerStateNode ssn = this.regionStates.getServerNode(serverName);
-    if (ssn == null) {
-      return true;
-    }
-    ServerState[] inState =
-      meta
-        ? new ServerState[] { ServerState.SPLITTING_META_DONE, ServerState.SPLITTING,
-          ServerState.OFFLINE }
-        : new ServerState[] { ServerState.OFFLINE };
-    synchronized (ssn) {
-      return ssn.isInState(inState);
-    }
-  }
-
-  /**
-   * Handle RIT of meta region against crashed server.
-   * Only used when ServerCrashProcedure is not enabled.
-   * See handleRIT in ServerCrashProcedure for similar function.
-   *
-   * @param serverName Server that has already crashed
-   */
-  public void handleMetaRITOnCrashedServer(ServerName serverName) {
-    RegionInfo hri = RegionReplicaUtil
-        .getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO,
-            RegionInfo.DEFAULT_REPLICA_ID);
-    RegionState regionStateNode = getRegionStates().getRegionState(hri);
-    if (regionStateNode == null) {
-      LOG.warn("RegionStateNode is null for " + hri);
-      return;
-    }
-    ServerName rsnServerName = regionStateNode.getServerName();
-    if (rsnServerName != null && !rsnServerName.equals(serverName)) {
-      return;
-    } else if (rsnServerName == null) {
-      LOG.warn("Empty ServerName in RegionStateNode; proceeding anyways in case latched " +
-          "RecoverMetaProcedure so meta latch gets cleaned up.");
-    }
-    // meta has been assigned to crashed server.
-    LOG.info("Meta assigned to crashed " + serverName + "; reassigning...");
-    // Handle failure and wake event
-    RegionTransitionProcedure rtp = getRegionStates().getRegionTransitionProcedure(hri);
-    // Do not need to consider for REGION_TRANSITION_QUEUE step
-    if (rtp != null && rtp.isMeta() &&
-        rtp.getTransitionState() == RegionTransitionState.REGION_TRANSITION_DISPATCH) {
-      LOG.debug("Failing " + rtp.toString());
-      rtp.remoteCallFailed(master.getMasterProcedureExecutor().getEnvironment(), serverName,
-          new ServerCrashException(rtp.getProcId(), serverName));
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
new file mode 100644
index 0000000..d0dca09
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManagerUtil.java
@@ -0,0 +1,195 @@
+/**
+ * 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.Collections;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.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
+final class AssignmentManagerUtil {
+  private AssignmentManagerUtil() {
+  }
+
+  /**
+   * 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 RegionInfo hri) throws IOException {
+    return getRegionInfoResponse(env, regionLocation, hri, false);
+  }
+
+  static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
+      final ServerName regionLocation, final RegionInfo hri, boolean includeBestSplitRow)
+      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 = null;
+    if (includeBestSplitRow) {
+      request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName(), false, true);
+    } else {
+      request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName());
+    }
+    try {
+      return admin.getRegionInfo(controller, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+
+  private static void lock(List<RegionStateNode> regionNodes) {
+    regionNodes.iterator().forEachRemaining(RegionStateNode::lock);
+  }
+
+  private static void unlock(List<RegionStateNode> regionNodes) {
+    for (ListIterator<RegionStateNode> iter = regionNodes.listIterator(regionNodes.size()); iter
+      .hasPrevious();) {
+      iter.previous().unlock();
+    }
+  }
+
+  static TransitRegionStateProcedure[] createUnassignProceduresForSplitOrMerge(
+      MasterProcedureEnv env, Stream<RegionInfo> regions, int regionReplication)
+      throws IOException {
+    List<RegionStateNode> regionNodes = regions
+      .flatMap(hri -> IntStream.range(0, regionReplication)
+        .mapToObj(i -> RegionReplicaUtil.getRegionInfoForReplica(hri, i)))
+      .map(env.getAssignmentManager().getRegionStates()::getOrCreateRegionStateNode)
+      .collect(Collectors.toList());
+    TransitRegionStateProcedure[] procs = new TransitRegionStateProcedure[regionNodes.size()];
+    boolean rollback = true;
+    int i = 0;
+    // hold the lock at once, and then release it in finally. This is important as SCP may jump in
+    // if we release the lock in the middle when we want to do rollback, and cause problems.
+    lock(regionNodes);
+    try {
+      for (; i < procs.length; i++) {
+        RegionStateNode regionNode = regionNodes.get(i);
+        TransitRegionStateProcedure proc =
+          TransitRegionStateProcedure.unassign(env, regionNode.getRegionInfo());
+        if (regionNode.getProcedure() != null) {
+          throw new HBaseIOException(
+            "The parent region " + regionNode + " is currently in transition, give up");
+        }
+        regionNode.setProcedure(proc);
+        procs[i] = proc;
+      }
+      // all succeeded, set rollback to false
+      rollback = false;
+    } finally {
+      if (rollback) {
+        for (;;) {
+          i--;
+          if (i < 0) {
+            break;
+          }
+          RegionStateNode regionNode = regionNodes.get(i);
+          regionNode.unsetProcedure(procs[i]);
+        }
+      }
+      unlock(regionNodes);
+    }
+    return procs;
+  }
+
+  private static TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env,
+      Stream<RegionInfo> regions, int regionReplication, ServerName targetServer) {
+    return regions
+      .flatMap(hri -> IntStream.range(0, regionReplication)
+        .mapToObj(i -> RegionReplicaUtil.getRegionInfoForReplica(hri, i)))
+      .map(env.getAssignmentManager().getRegionStates()::getOrCreateRegionStateNode)
+      .map(regionNode -> {
+        TransitRegionStateProcedure proc =
+          TransitRegionStateProcedure.assign(env, regionNode.getRegionInfo(), targetServer);
+        regionNode.lock();
+        try {
+          // should never fail, as we have the exclusive region lock, and the region is newly
+          // created, or has been successfully closed so should not be on any servers, so SCP will
+          // not process it either.
+          assert !regionNode.isInTransition();
+          regionNode.setProcedure(proc);
+        } finally {
+          regionNode.unlock();
+        }
+        return proc;
+      }).toArray(TransitRegionStateProcedure[]::new);
+  }
+
+  static TransitRegionStateProcedure[] createAssignProceduresForOpeningNewRegions(
+      MasterProcedureEnv env, Stream<RegionInfo> regions, int regionReplication,
+      ServerName targetServer) {
+    return createAssignProcedures(env, regions, regionReplication, targetServer);
+  }
+
+  static void reopenRegionsForRollback(MasterProcedureEnv env, Stream<RegionInfo> regions,
+      int regionReplication, ServerName targetServer) {
+    TransitRegionStateProcedure[] procs =
+      createAssignProcedures(env, regions, regionReplication, targetServer);
+    env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
+  }
+
+  static void removeNonDefaultReplicas(MasterProcedureEnv env, Stream<RegionInfo> regions,
+      int regionReplication) {
+    // Remove from in-memory states
+    regions.flatMap(hri -> IntStream.range(1, regionReplication)
+      .mapToObj(i -> RegionReplicaUtil.getRegionInfoForReplica(hri, i))).forEach(hri -> {
+        env.getAssignmentManager().getRegionStates().deleteRegion(hri);
+        env.getMasterServices().getServerManager().removeRegion(hri);
+        FavoredNodesManager fnm = env.getMasterServices().getFavoredNodesManager();
+        if (fnm != null) {
+          fnm.deleteFavoredNodesForRegions(Collections.singletonList(hri));
+        }
+      });
+  }
+
+  static void checkClosedRegion(MasterProcedureEnv env, RegionInfo regionInfo) throws IOException {
+    if (WALSplitter.hasRecoveredEdits(env.getMasterServices().getFileSystem(),
+      env.getMasterConfiguration(), regionInfo)) {
+      throw new IOException("Recovered.edits are found in Region: " + regionInfo +
+        ", abort split/merge to prevent data loss");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
new file mode 100644
index 0000000..e446e17
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/CloseRegionProcedure.java
@@ -0,0 +1,82 @@
+/**
+ * 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.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloseRegionProcedureStateData;
+
+/**
+ * The remote procedure used to close a region.
+ */
+@InterfaceAudience.Private
+public class CloseRegionProcedure extends RegionRemoteProcedureBase {
+
+  // For a region move operation, we will assign the region after we unassign it, this is the target
+  // server for the subsequent assign. We will send this value to RS, and RS will record the region
+  // in a Map to tell client that where the region has been moved to. Can be null. And also, can be
+  // wrong(but do not make it wrong intentionally). The client can handle this error.
+  private ServerName assignCandidate;
+
+  public CloseRegionProcedure() {
+    super();
+  }
+
+  public CloseRegionProcedure(RegionInfo region, ServerName targetServer,
+      ServerName assignCandidate) {
+    super(region, targetServer);
+    this.assignCandidate = assignCandidate;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REGION_UNASSIGN;
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    return new RegionCloseOperation(this, region, assignCandidate);
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    CloseRegionProcedureStateData.Builder builder = CloseRegionProcedureStateData.newBuilder();
+    if (assignCandidate != null) {
+      builder.setAssignCandidate(ProtobufUtil.toServerName(assignCandidate));
+    }
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    CloseRegionProcedureStateData data =
+      serializer.deserialize(CloseRegionProcedureStateData.class);
+    if (data.hasAssignCandidate()) {
+      assignCandidate = ProtobufUtil.toServerName(data.getAssignCandidate());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 20ae444..ff2ba5b 100644
--- 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
@@ -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
@@ -15,7 +15,6 @@
  * 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;
@@ -23,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-
+import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -64,7 +62,9 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@@ -72,10 +72,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.M
 
 /**
  * The procedure to Merge a region in a table.
+ * <p/>
  * This procedure takes an exclusive table lock since it is working over multiple regions.
+ * <p/>
  * It holds the lock for the life of the procedure.
- * <p>Throws exception on construction if determines context hostile to merge (cluster going
- * down or master is shutting down or table is disabled).</p>
+ * <p/>
+ * Throws exception on construction if determines context hostile to merge (cluster going down or
+ * master is shutting down or table is disabled).
  */
 @InterfaceAudience.Private
 public class MergeTableRegionsProcedure
@@ -216,6 +219,20 @@ public class MergeTableRegionsProcedure
     return rid;
   }
 
+
+  private void removeNonDefaultReplicas(MasterProcedureEnv env) throws IOException {
+    AssignmentManagerUtil.removeNonDefaultReplicas(env, Stream.of(regionsToMerge),
+      getRegionReplication(env));
+  }
+
+  private void checkClosedRegions(MasterProcedureEnv env) throws IOException {
+    // theoretically this should not happen any more after we use TRSP, but anyway let's add a check
+    // here
+    for (RegionInfo region : regionsToMerge) {
+      AssignmentManagerUtil.checkClosedRegion(env, region);
+    }
+  }
+
   @Override
   protected Flow executeFromState(final MasterProcedureEnv env,
       MergeTableRegionsState state) {
@@ -234,27 +251,15 @@ public class MergeTableRegionsProcedure
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CLOSE_REGIONS);
           break;
         case MERGE_TABLE_REGIONS_CLOSE_REGIONS:
-          addChildProcedure(createUnassignProcedures(env, getRegionReplication(env)));
+          addChildProcedure(createUnassignProcedures(env));
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS);
           break;
         case MERGE_TABLE_REGIONS_CHECK_CLOSED_REGIONS:
-          List<RegionInfo> ris = hasRecoveredEdits(env);
-          if (ris.isEmpty()) {
-            setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
-          } else {
-            // Need to reopen parent regions to pickup missed recovered.edits. Do it by creating
-            // child assigns and then stepping back to MERGE_TABLE_REGIONS_CLOSE_REGIONS.
-            // Just assign the primary regions recovering the missed recovered.edits -- no replicas.
-            // May need to cycle here a few times if heavy writes.
-            // TODO: Add an assign read-only.
-            for (RegionInfo ri: ris) {
-              LOG.info("Found recovered.edits under {}, reopen to pickup missed edits!", ri);
-              addChildProcedure(env.getAssignmentManager().createAssignProcedure(ri));
-            }
-            setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CLOSE_REGIONS);
-          }
+          checkClosedRegions(env);
+          setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_CREATE_MERGED_REGION);
           break;
         case MERGE_TABLE_REGIONS_CREATE_MERGED_REGION:
+          removeNonDefaultReplicas(env);
           createMergedRegion(env);
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_WRITE_MAX_SEQUENCE_ID_FILE);
           break;
@@ -275,7 +280,7 @@ public class MergeTableRegionsProcedure
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_OPEN_MERGED_REGION);
           break;
         case MERGE_TABLE_REGIONS_OPEN_MERGED_REGION:
-          addChildProcedure(createAssignProcedures(env, getRegionReplication(env)));
+          addChildProcedure(createAssignProcedures(env));
           setNextState(MergeTableRegionsState.MERGE_TABLE_REGIONS_POST_OPERATION);
           break;
         case MERGE_TABLE_REGIONS_POST_OPERATION:
@@ -471,25 +476,7 @@ public class MergeTableRegionsProcedure
   }
 
   /**
-   * Return list of regions that have recovered.edits... usually its an empty list.
-   * @param env the master env
-   * @throws IOException IOException
-   */
-  private List<RegionInfo> hasRecoveredEdits(final MasterProcedureEnv env) throws IOException {
-    List<RegionInfo> ris =  new ArrayList<RegionInfo>(regionsToMerge.length);
-    for (int i = 0; i < regionsToMerge.length; i++) {
-      RegionInfo ri = regionsToMerge[i];
-      if (SplitTableRegionProcedure.hasRecoveredEdits(env, ri)) {
-        ris.add(ri);
-      }
-    }
-    return ris;
-  }
-
-  /**
    * 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,
@@ -559,9 +546,9 @@ public class MergeTableRegionsProcedure
   }
 
   private boolean isMergeable(final MasterProcedureEnv env, final RegionState rs)
-  throws IOException {
+      throws IOException {
     GetRegionInfoResponse response =
-      Util.getRegionInfoResponse(env, rs.getServerName(), rs.getRegion());
+      AssignmentManagerUtil.getRegionInfoResponse(env, rs.getServerName(), rs.getRegion());
     return response.hasMergeable() && response.getMergeable();
   }
 
@@ -598,9 +585,8 @@ public class MergeTableRegionsProcedure
 
   /**
    * Set the region states to MERGING state
-   * @param env MasterProcedureEnv
    */
-  public void setRegionStateToMerging(final MasterProcedureEnv env) {
+  private void setRegionStateToMerging(final MasterProcedureEnv env) {
     // Set State.MERGING to regions to be merged
     RegionStates regionStates = env.getAssignmentManager().getRegionStates();
     regionStates.getRegionStateNode(regionsToMerge[0]).setState(State.MERGING);
@@ -675,49 +661,22 @@ public class MergeTableRegionsProcedure
 
   /**
    * 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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
-        procs[procsIdx++] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
-      }
-    }
-    env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
+  private void rollbackCloseRegionsForMerge(MasterProcedureEnv env) throws IOException {
+    AssignmentManagerUtil.reopenRegionsForRollback(env, Stream.of(regionsToMerge),
+      getRegionReplication(env), getServerName(env));
   }
 
-  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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(regionsToMerge[i], j);
-        procs[procsIdx++] = env.getAssignmentManager().
-            createUnassignProcedure(hri, null, true, !RegionReplicaUtil.isDefaultReplica(hri));
-      }
-    }
-    return procs;
+  private TransitRegionStateProcedure[] createUnassignProcedures(MasterProcedureEnv env)
+      throws IOException {
+    return AssignmentManagerUtil.createUnassignProceduresForSplitOrMerge(env,
+      Stream.of(regionsToMerge), getRegionReplication(env));
   }
 
-  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 RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(mergedRegion, i);
-      procs[i] = env.getAssignmentManager().createAssignProcedure(hri, targetServer);
-    }
-    return procs;
+  private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env)
+      throws IOException {
+    return AssignmentManagerUtil.createAssignProceduresForOpeningNewRegions(env,
+      Stream.of(mergedRegion), getRegionReplication(env), getServerName(env));
   }
 
   private int getRegionReplication(final MasterProcedureEnv env) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 6135ce1..3aadb92 100644
--- 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
@@ -1,5 +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
@@ -20,8 +19,6 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
-
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -30,81 +27,29 @@ import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProced
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 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.
- *
- * <p>Throws exception on construction if determines context hostile to move (cluster going
- * down or master is shutting down or table is disabled).</p>
+ * Leave here only for checking if we can successfully start the master.
+ * @deprecated Do not use any more.
+ * @see TransitRegionStateProcedure
  */
+@Deprecated
 @InterfaceAudience.Private
 public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<MoveRegionState> {
-  private static final Logger LOG = LoggerFactory.getLogger(MoveRegionProcedure.class);
   private RegionPlan plan;
 
   public MoveRegionProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
     super();
   }
 
-  /**
-   * @param check whether we should do some checks in the constructor. We will skip the checks if we
-   *          are reopening a region as this may fail the whole procedure and cause stuck. We will
-   *          do the check later when actually executing the procedure so not a big problem.
-   * @throws IOException If the cluster is offline or master is stopping or if table is disabled or
-   *           non-existent.
-   */
-  public MoveRegionProcedure(MasterProcedureEnv env, RegionPlan plan, boolean check)
-      throws HBaseIOException {
-    super(env, plan.getRegionInfo());
-    this.plan = plan;
-    if (check) {
-      preflightChecks(env, true);
-      checkOnline(env, plan.getRegionInfo());
-    }
-  }
-
   @Override
   protected Flow executeFromState(final MasterProcedureEnv env, final MoveRegionState state)
       throws InterruptedException {
-    LOG.trace("{} execute state={}", this, state);
-    switch (state) {
-      case MOVE_REGION_PREPARE:
-        // Check context again and that region is online; do it here after we have lock on region.
-        try {
-          preflightChecks(env, true);
-          checkOnline(env, this.plan.getRegionInfo());
-          if (!env.getMasterServices().getServerManager().isServerOnline(this.plan.getSource())) {
-            throw new HBaseIOException(this.plan.getSource() + " not online");
-          }
-        } catch (HBaseIOException e) {
-          LOG.warn(this.toString() + " FAILED because " + e.toString());
-          return Flow.NO_MORE_STATE;
-        }
-        break;
-      case MOVE_REGION_UNASSIGN:
-        addChildProcedure(new UnassignProcedure(plan.getRegionInfo(), plan.getSource(),
-            plan.getDestination(), true));
-        setNextState(MoveRegionState.MOVE_REGION_ASSIGN);
-        break;
-      case MOVE_REGION_ASSIGN:
-        AssignProcedure assignProcedure = plan.getDestination() == null ?
-            new AssignProcedure(plan.getRegionInfo()):
-            new AssignProcedure(plan.getRegionInfo(), plan.getDestination());
-        addChildProcedure(assignProcedure);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException("unhandled state=" + state);
-    }
-    return Flow.HAS_MORE_STATE;
+    return Flow.NO_MORE_STATE;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
new file mode 100644
index 0000000..1a79697
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/OpenRegionProcedure.java
@@ -0,0 +1,67 @@
+/**
+ * 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.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionOpenOperation;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.OpenRegionProcedureStateData;
+
+/**
+ * The remote procedure used to open a region.
+ */
+@InterfaceAudience.Private
+public class OpenRegionProcedure extends RegionRemoteProcedureBase {
+
+  public OpenRegionProcedure() {
+    super();
+  }
+
+  public OpenRegionProcedure(RegionInfo region, ServerName targetServer) {
+    super(region, targetServer);
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REGION_ASSIGN;
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    return new RegionOpenOperation(this, region, env.getAssignmentManager().getFavoredNodes(region),
+      false);
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(OpenRegionProcedureStateData.getDefaultInstance());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    serializer.deserialize(OpenRegionProcedureStateData.class);
+  }
+}


[3/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 775c8c2..db7a872 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
@@ -20,18 +20,17 @@ package org.apache.hadoop.hbase.master.procedure;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.MasterWalManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.assignment.RegionTransitionProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
@@ -98,7 +97,10 @@ public class ServerCrashProcedure
    * #deserializeStateData(InputStream). Do not use directly.
    */
   public ServerCrashProcedure() {
-    super();
+  }
+
+  public boolean isInRecoverMetaState() {
+    return getCurrentState() == ServerCrashState.SERVER_CRASH_PROCESS_META;
   }
 
   @Override
@@ -128,15 +130,7 @@ public class ServerCrashProcedure
           setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_META);
           break;
         case SERVER_CRASH_ASSIGN_META:
-          handleRIT(env, Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO));
-          addChildProcedure(env.getAssignmentManager()
-            .createAssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
-          setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
-          break;
-        case SERVER_CRASH_PROCESS_META:
-          // not used any more but still leave it here to keep compatible as there maybe old SCP
-          // which is stored in ProcedureStore which has this state.
-          processMeta(env);
+          assignRegions(env, Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO));
           setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
           break;
         case SERVER_CRASH_GET_REGIONS:
@@ -144,8 +138,8 @@ public class ServerCrashProcedure
           if (env.getAssignmentManager().waitMetaLoaded(this)) {
             throw new ProcedureSuspendedException();
           }
-          this.regionsOnCrashedServer = services.getAssignmentManager().getRegionStates()
-            .getServerRegionInfoSet(serverName);
+          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.
           if (!this.shouldSplitWal) {
@@ -162,26 +156,15 @@ public class ServerCrashProcedure
           // 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 (filterDefaultMetaRegions()) {
             if (LOG.isTraceEnabled()) {
-              LOG.trace("Assigning regions " +
-                RegionInfo.getShortNameToLog(regionsOnCrashedServer) + ", " + this +
-                "; cycles=" + getCycles());
+              LOG
+                .trace("Assigning regions " + RegionInfo.getShortNameToLog(regionsOnCrashedServer) +
+                  ", " + this + "; cycles=" + getCycles());
             }
-            // Handle RIT against crashed server. Will cancel any ongoing assigns/unassigns.
-            // Returns list of regions we need to reassign.
-            // NOTE: there is nothing to stop a dispatch happening AFTER this point. Check for the
-            // condition if a dispatch RPC fails inside in AssignProcedure/UnassignProcedure.
-            // AssignProcedure just keeps retrying. UnassignProcedure is more complicated. See where
-            // it does the check by calling am#isLogSplittingDone.
-            List<RegionInfo> toAssign = handleRIT(env, regionsOnCrashedServer);
-            AssignmentManager am = env.getAssignmentManager();
-            // CreateAssignProcedure will try to use the old location for the region deploy.
-            addChildProcedure(am.createAssignProcedures(toAssign));
-            setNextState(ServerCrashState.SERVER_CRASH_HANDLE_RIT2);
-          } else {
-            setNextState(ServerCrashState.SERVER_CRASH_FINISH);
+            assignRegions(env, regionsOnCrashedServer);
           }
+          setNextState(ServerCrashState.SERVER_CRASH_FINISH);
           break;
         case SERVER_CRASH_HANDLE_RIT2:
           // Noop. Left in place because we used to call handleRIT here for a second time
@@ -201,28 +184,16 @@ public class ServerCrashProcedure
     return Flow.HAS_MORE_STATE;
   }
 
-  private void processMeta(final MasterProcedureEnv env) throws IOException {
-    LOG.debug("{}; processing hbase:meta", this);
-
-    // Assign meta if still carrying it. Check again: region may be assigned because of RIT timeout
-    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    for (RegionInfo hri: am.getRegionStates().getServerRegionInfoSet(serverName)) {
-      if (!isDefaultMetaRegion(hri)) {
-        continue;
-      }
-      addChildProcedure(new RecoverMetaProcedure(serverName, this.shouldSplitWal));
+  private boolean filterDefaultMetaRegions() {
+    if (regionsOnCrashedServer == null) {
+      return false;
     }
+    regionsOnCrashedServer.removeIf(this::isDefaultMetaRegion);
+    return !regionsOnCrashedServer.isEmpty();
   }
 
-  private boolean filterDefaultMetaRegions(final List<RegionInfo> regions) {
-    if (regions == null) return false;
-    regions.removeIf(this::isDefaultMetaRegion);
-    return !regions.isEmpty();
-  }
-
-  private boolean isDefaultMetaRegion(final RegionInfo hri) {
-    return hri.getTable().equals(TableName.META_TABLE_NAME) &&
-      RegionReplicaUtil.isDefaultReplica(hri);
+  private boolean isDefaultMetaRegion(RegionInfo hri) {
+    return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
   }
 
   private void splitMetaLogs(MasterProcedureEnv env) throws IOException {
@@ -372,54 +343,37 @@ public class ServerCrashProcedure
   }
 
   /**
-   * 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. If unable to unassign because server went away, unassigns block waiting
-   * on the below callback from a ServerCrashProcedure before proceeding.
-   * @param regions Regions on the Crashed Server.
-   * @return List of regions we should assign to new homes (not same as regions on crashed server).
+   * Assign the regions on the crashed RS to other Rses.
+   * <p/>
+   * In this method we will go through all the RegionStateNodes of the give regions to find out
+   * whether there is already an TRSP for the region, if so we interrupt it and let it retry on
+   * other server, otherwise we will schedule a TRSP to bring the region online.
+   * <p/>
+   * We will also check whether the table for a region is enabled, if not, we will skip assigning
+   * it.
    */
-  private List<RegionInfo> handleRIT(final MasterProcedureEnv env, List<RegionInfo> regions) {
-    if (regions == null || regions.isEmpty()) {
-      return Collections.emptyList();
-    }
+  private void assignRegions(MasterProcedureEnv env, List<RegionInfo> regions) throws IOException {
     AssignmentManager am = env.getMasterServices().getAssignmentManager();
-    List<RegionInfo> toAssign = new ArrayList<RegionInfo>(regions);
-    // Get an iterator so can remove items.
-    final Iterator<RegionInfo> it = toAssign.iterator();
-    ServerCrashException sce = null;
-    while (it.hasNext()) {
-      final RegionInfo 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());
+    for (RegionInfo region : regions) {
+      RegionStateNode regionNode = am.getRegionStates().getOrCreateRegionStateNode(region);
+      regionNode.lock();
+      try {
+        if (regionNode.getProcedure() != null) {
+          LOG.info("{} found RIT {}; {}", this, regionNode.getProcedure(), regionNode);
+          regionNode.getProcedure().serverCrashed(env, regionNode, getServerName());
+        } else {
+          if (env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),
+            TableState.State.DISABLING, TableState.State.DISABLED)) {
+            continue;
+          }
+          TransitRegionStateProcedure proc = TransitRegionStateProcedure.assign(env, region, null);
+          regionNode.setProcedure(proc);
+          addChildProcedure(proc);
+        }
+      } finally {
+        regionNode.unlock();
       }
-      rtp.remoteCallFailed(env, this.serverName, sce);
-      // If an assign, remove from passed-in list of regions so we subsequently do not create
-      // a new assign; the exisitng assign after the call to remoteCallFailed will recalibrate
-      // and assign to a server other than the crashed one; no need to create new assign.
-      // If an unassign, do not return this region; the above cancel will wake up the unassign and
-      // it will complete. Done.
-      it.remove();
     }
-    return toAssign;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 d6c8607..52da607 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
@@ -189,7 +189,7 @@ public class TruncateTableProcedure
 
   @Override
   protected TruncateTableState getState(final int stateId) {
-    return TruncateTableState.valueOf(stateId);
+    return TruncateTableState.forNumber(stateId);
   }
 
   @Override
@@ -203,6 +203,11 @@ public class TruncateTableProcedure
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  @Override
   public TableName getTableName() {
     return tableName;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
deleted file mode 100644
index 1f22830..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ /dev/null
@@ -1,294 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
-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.RegionInfo;
-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;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category({MasterTests.class, MediumTests.class})
-public class TestAssignmentListener {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAssignmentListener.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentListener.class);
-
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
-  @Rule
-  public TestName name = new TestName();
-
-  static class DummyListener {
-    protected AtomicInteger modified = new AtomicInteger(0);
-
-    public void awaitModifications(int count) throws InterruptedException {
-      while (!modified.compareAndSet(count, 0)) {
-        Thread.sleep(100);
-      }
-    }
-  }
-
-  static class DummyAssignmentListener extends DummyListener implements AssignmentListener {
-    private AtomicInteger closeCount = new AtomicInteger(0);
-    private AtomicInteger openCount = new AtomicInteger(0);
-
-    public DummyAssignmentListener() {
-    }
-
-    @Override
-    public void regionOpened(final RegionInfo regionInfo, final ServerName serverName) {
-      LOG.info("Assignment open region=" + regionInfo + " server=" + serverName);
-      openCount.incrementAndGet();
-      modified.incrementAndGet();
-    }
-
-    @Override
-    public void regionClosed(final RegionInfo regionInfo) {
-      LOG.info("Assignment close region=" + regionInfo);
-      closeCount.incrementAndGet();
-      modified.incrementAndGet();
-    }
-
-    public void reset() {
-      openCount.set(0);
-      closeCount.set(0);
-    }
-
-    public int getLoadCount() {
-      return openCount.get();
-    }
-
-    public int getCloseCount() {
-      return closeCount.get();
-    }
-  }
-
-  static class DummyServerListener extends DummyListener implements ServerListener {
-    private AtomicInteger removedCount = new AtomicInteger(0);
-    private AtomicInteger addedCount = new AtomicInteger(0);
-
-    public DummyServerListener() {
-    }
-
-    @Override
-    public void serverAdded(final ServerName serverName) {
-      LOG.info("Server added " + serverName);
-      addedCount.incrementAndGet();
-      modified.incrementAndGet();
-    }
-
-    @Override
-    public void serverRemoved(final ServerName serverName) {
-      LOG.info("Server removed " + serverName);
-      removedCount.incrementAndGet();
-      modified.incrementAndGet();
-    }
-
-    public void reset() {
-      addedCount.set(0);
-      removedCount.set(0);
-    }
-
-    public int getAddedCount() {
-      return addedCount.get();
-    }
-
-    public int getRemovedCount() {
-      return removedCount.get();
-    }
-  }
-
-  @BeforeClass
-  public static void beforeAllTests() throws Exception {
-    TEST_UTIL.startMiniCluster(2);
-  }
-
-  @AfterClass
-  public static void afterAllTests() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testServerListener() throws IOException, InterruptedException {
-    ServerManager serverManager = TEST_UTIL.getHBaseCluster().getMaster().getServerManager();
-
-    DummyServerListener listener = new DummyServerListener();
-    serverManager.registerListener(listener);
-    try {
-      MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
-
-      // Start a new Region Server
-      miniCluster.startRegionServer();
-      listener.awaitModifications(1);
-      assertEquals(1, listener.getAddedCount());
-      assertEquals(0, listener.getRemovedCount());
-
-      // Start another Region Server
-      listener.reset();
-      miniCluster.startRegionServer();
-      listener.awaitModifications(1);
-      assertEquals(1, listener.getAddedCount());
-      assertEquals(0, listener.getRemovedCount());
-
-      int nrs = miniCluster.getRegionServerThreads().size();
-
-      // Stop a Region Server
-      listener.reset();
-      miniCluster.stopRegionServer(nrs - 1);
-      listener.awaitModifications(1);
-      assertEquals(0, listener.getAddedCount());
-      assertEquals(1, listener.getRemovedCount());
-
-      // Stop another Region Server
-      listener.reset();
-      miniCluster.stopRegionServer(nrs - 2);
-      listener.awaitModifications(1);
-      assertEquals(0, listener.getAddedCount());
-      assertEquals(1, listener.getRemovedCount());
-    } finally {
-      serverManager.unregisterListener(listener);
-    }
-  }
-
-  @Test
-  public void testAssignmentListener() throws IOException, InterruptedException {
-    AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
-    Admin admin = TEST_UTIL.getAdmin();
-
-    DummyAssignmentListener listener = new DummyAssignmentListener();
-    am.registerListener(listener);
-    try {
-      final TableName tableName = TableName.valueOf(name.getMethodName());
-      final byte[] FAMILY = Bytes.toBytes("cf");
-
-      // Create a new table, with a single region
-      LOG.info("Create Table");
-      TEST_UTIL.createTable(tableName, FAMILY);
-      listener.awaitModifications(1);
-      assertEquals(1, listener.getLoadCount());
-      assertEquals(0, listener.getCloseCount());
-
-      // Add some data
-      Table table = TEST_UTIL.getConnection().getTable(tableName);
-      try {
-        for (int i = 0; i < 10; ++i) {
-          byte[] key = Bytes.toBytes("row-" + i);
-          Put put = new Put(key);
-          put.addColumn(FAMILY, null, key);
-          table.put(put);
-        }
-      } finally {
-        table.close();
-      }
-
-      // Split the table in two
-      LOG.info("Split Table");
-      listener.reset();
-      admin.split(tableName, Bytes.toBytes("row-3"));
-      listener.awaitModifications(3);
-      assertEquals(2, listener.getLoadCount());     // daughters added
-      assertEquals(1, listener.getCloseCount());    // parent removed
-
-      // Wait for the Regions to be mergeable
-      MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
-      int mergeable = 0;
-      while (mergeable < 2) {
-        Thread.sleep(100);
-        admin.majorCompact(tableName);
-        mergeable = 0;
-        for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
-          for (Region region: regionThread.getRegionServer().getRegions(tableName)) {
-            mergeable += ((HRegion)region).isMergeable() ? 1 : 0;
-          }
-        }
-      }
-
-      // Merge the two regions
-      LOG.info("Merge Regions");
-      listener.reset();
-      List<RegionInfo> regions = admin.getRegions(tableName);
-      assertEquals(2, regions.size());
-      boolean sameServer = areAllRegionsLocatedOnSameServer(tableName);
-      // If the regions are located by different server, we need to move
-      // regions to same server before merging. So the expected modifications
-      // will increaes to 5. (open + close)
-      final int expectedModifications = sameServer ? 3 : 5;
-      final int expectedLoadCount = sameServer ? 1 : 2;
-      final int expectedCloseCount = sameServer ? 2 : 3;
-      admin.mergeRegionsAsync(regions.get(0).getEncodedNameAsBytes(),
-        regions.get(1).getEncodedNameAsBytes(), true);
-      listener.awaitModifications(expectedModifications);
-      assertEquals(1, admin.getRegions(tableName).size());
-      assertEquals(expectedLoadCount, listener.getLoadCount());     // new merged region added
-      assertEquals(expectedCloseCount, listener.getCloseCount());    // daughters removed
-
-      // Delete the table
-      LOG.info("Drop Table");
-      listener.reset();
-      TEST_UTIL.deleteTable(tableName);
-      listener.awaitModifications(1);
-      assertEquals(0, listener.getLoadCount());
-      assertEquals(1, listener.getCloseCount());
-    } finally {
-      am.unregisterListener(listener);
-    }
-  }
-
-  private boolean areAllRegionsLocatedOnSameServer(TableName TABLE_NAME) {
-    MiniHBaseCluster miniCluster = TEST_UTIL.getMiniHBaseCluster();
-    int serverCount = 0;
-    for (JVMClusterUtil.RegionServerThread regionThread: miniCluster.getRegionServerThreads()) {
-      if (!regionThread.getRegionServer().getRegions(TABLE_NAME).isEmpty()) {
-        ++serverCount;
-      }
-      if (serverCount > 1) {
-        return false;
-      }
-    }
-    return serverCount == 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
index 41a8001..3df4929 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterAbortAndRSGotKilled.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.Optional;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -30,12 +29,14 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.master.assignment.MoveRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 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.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.Threads;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -45,15 +46,14 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 @Category({ MasterTests.class, MediumTests.class })
 public class TestMasterAbortAndRSGotKilled {
-  private static Logger LOG = LoggerFactory
-      .getLogger(TestMasterAbortAndRSGotKilled.class.getName());
+  private static Logger LOG =
+    LoggerFactory.getLogger(TestMasterAbortAndRSGotKilled.class.getName());
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMasterAbortAndRSGotKilled.class);
+    HBaseClassTestRule.forClass(TestMasterAbortAndRSGotKilled.class);
 
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
@@ -61,14 +61,12 @@ public class TestMasterAbortAndRSGotKilled {
 
   private static CountDownLatch countDownLatch = new CountDownLatch(1);
 
-
-
   private static byte[] CF = Bytes.toBytes("cf");
 
   @BeforeClass
   public static void setUp() throws Exception {
     UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
-        DelayCloseCP.class.getName());
+      DelayCloseCP.class.getName());
     UTIL.startMiniCluster(3);
     UTIL.getAdmin().balancerSwitch(false, true);
     UTIL.createTable(TABLE_NAME, CF);
@@ -84,48 +82,44 @@ public class TestMasterAbortAndRSGotKilled {
   public void test() throws Exception {
     JVMClusterUtil.RegionServerThread rsThread = null;
     for (JVMClusterUtil.RegionServerThread t : UTIL.getMiniHBaseCluster()
-        .getRegionServerThreads()) {
+      .getRegionServerThreads()) {
       if (!t.getRegionServer().getRegions(TABLE_NAME).isEmpty()) {
         rsThread = t;
         break;
       }
     }
-    //find the rs and hri of the table
+    // find the rs and hri of the table
     HRegionServer rs = rsThread.getRegionServer();
     RegionInfo hri = rs.getRegions(TABLE_NAME).get(0).getRegionInfo();
-    MoveRegionProcedure moveRegionProcedure = new MoveRegionProcedure(
-      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
-        .getEnvironment(),
-        new RegionPlan(hri, rs.getServerName(), rs.getServerName()), true);
-    long procID = UTIL.getMiniHBaseCluster().getMaster()
-      .getMasterProcedureExecutor().submitProcedure(moveRegionProcedure);
+    TransitRegionStateProcedure moveRegionProcedure = TransitRegionStateProcedure.reopen(
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getEnvironment(), hri);
+    RegionStateNode regionNode = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
+      .getRegionStates().getOrCreateRegionStateNode(hri);
+    regionNode.setProcedure(moveRegionProcedure);
+    UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
+      .submitProcedure(moveRegionProcedure);
     countDownLatch.await();
     UTIL.getMiniHBaseCluster().stopMaster(0);
     UTIL.getMiniHBaseCluster().startMaster();
-    //wait until master initialized
-    UTIL.waitFor(30000,
-      () -> UTIL.getMiniHBaseCluster().getMaster() != null && UTIL
-        .getMiniHBaseCluster().getMaster().isInitialized());
+    // wait until master initialized
+    UTIL.waitFor(30000, () -> UTIL.getMiniHBaseCluster().getMaster() != null &&
+      UTIL.getMiniHBaseCluster().getMaster().isInitialized());
     Assert.assertTrue("Should be 3 RS after master restart",
-        UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size() == 3);
+      UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size() == 3);
 
   }
 
-  public static class DelayCloseCP implements RegionCoprocessor,
-      RegionObserver {
-    @Override
-    public void preClose(ObserverContext<RegionCoprocessorEnvironment> c,
-        boolean abortRequested) throws IOException {
-      try {
-        if (!c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
-          LOG.error("begin to sleep");
-          countDownLatch.countDown();
-          //Sleep here so we can stuck the RPC call
-          Thread.sleep(10000);
-          LOG.error("finish sleep");
-        }
-      } catch (Throwable t) {
+  public static class DelayCloseCP implements RegionCoprocessor, RegionObserver {
 
+    @Override
+    public void preClose(ObserverContext<RegionCoprocessorEnvironment> c, boolean abortRequested)
+        throws IOException {
+      if (!c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
+        LOG.info("begin to sleep");
+        countDownLatch.countDown();
+        // Sleep here so we can stuck the RPC call
+        Threads.sleep(10000);
+        LOG.info("finish sleep");
       }
     }
 
@@ -134,5 +128,4 @@ public class TestMasterAbortAndRSGotKilled {
       return Optional.of(this);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
index 9608e5c..7cf794a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTableRegionsWhileRSCrash.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master;
 
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
-import org.apache.hadoop.hbase.master.assignment.UnassignProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -103,9 +102,9 @@ public class TestMergeTableRegionsWhileRSCrash {
     MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
         env, regionInfos.get(0), regionInfos.get(1));
     executor.submitProcedure(mergeTableRegionsProcedure);
-    UTIL.waitFor(30000, () -> executor.getProcedures().stream()
-        .filter(p -> p instanceof UnassignProcedure)
-        .map(p -> (UnassignProcedure) p)
+    UTIL.waitFor(30000,
+      () -> executor.getProcedures().stream().filter(p -> p instanceof TransitRegionStateProcedure)
+        .map(p -> (TransitRegionStateProcedure) p)
         .anyMatch(p -> TABLE_NAME.equals(p.getTableName())));
     UTIL.getMiniHBaseCluster().killRegionServer(
         UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java
index 748cd0e..88cde00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.AsyncAdmin;
 import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -83,9 +83,10 @@ public class TestServerCrashProcedureCarryingMetaStuck {
       rs.abort("For testing!");
 
       UTIL.waitFor(30000,
-        () -> executor.getProcedures().stream().filter(p -> p instanceof AssignProcedure)
-          .map(p -> (AssignProcedure) p)
-          .anyMatch(p -> Bytes.equals(hri.getRegionName(), p.getRegionInfo().getRegionName())));
+        () -> executor.getProcedures().stream()
+          .filter(p -> p instanceof TransitRegionStateProcedure)
+          .map(p -> (TransitRegionStateProcedure) p)
+          .anyMatch(p -> Bytes.equals(hri.getRegionName(), p.getRegion().getRegionName())));
       proc.resume();
       UTIL.waitFor(30000, () -> executor.isFinished(procId));
       // see whether the move region procedure can finish properly

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureStuck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureStuck.java
index 2681657..b6dedbe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureStuck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureStuck.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.AsyncAdmin;
 import org.apache.hadoop.hbase.client.AsyncConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -92,9 +92,10 @@ public class TestServerCrashProcedureStuck {
       rs.abort("For testing!");
 
       UTIL.waitFor(30000,
-        () -> executor.getProcedures().stream().filter(p -> p instanceof AssignProcedure)
-          .map(p -> (AssignProcedure) p)
-          .anyMatch(p -> Bytes.equals(hri.getRegionName(), p.getRegionInfo().getRegionName())));
+        () -> executor.getProcedures().stream()
+          .filter(p -> p instanceof TransitRegionStateProcedure)
+          .map(p -> (TransitRegionStateProcedure) p)
+          .anyMatch(p -> Bytes.equals(hri.getRegionName(), p.getRegion().getRegionName())));
       proc.resume();
       UTIL.waitFor(30000, () -> executor.isFinished(procId));
       // see whether the move region procedure can finish properly

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
index a881575..fe5d1a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitRegionWhileRSCrash.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master;
 
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -31,7 +30,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.assignment.SplitTableRegionProcedure;
-import org.apache.hadoop.hbase.master.assignment.UnassignProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -103,8 +102,8 @@ public class TestSplitRegionWhileRSCrash {
     executor.submitProcedure(splitProcedure);
     LOG.info("SplitProcedure submitted");
     UTIL.waitFor(30000, () -> executor.getProcedures().stream()
-        .filter(p -> p instanceof UnassignProcedure)
-        .map(p -> (UnassignProcedure) p)
+        .filter(p -> p instanceof TransitRegionStateProcedure)
+        .map(p -> (TransitRegionStateProcedure) p)
         .anyMatch(p -> TABLE_NAME.equals(p.getTableName())));
     UTIL.getMiniHBaseCluster().killRegionServer(
         UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index c4a2f03..3206877 100644
--- 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
@@ -319,7 +319,7 @@ public class MockMasterServices extends MockNoopMasterServices {
     }
 
     @Override
-    public void updateRegionLocation(RegionStates.RegionStateNode regionNode) throws IOException {
+    public void updateRegionLocation(RegionStateNode regionNode) throws IOException {
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMAssignWithRandExec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMAssignWithRandExec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMAssignWithRandExec.java
new file mode 100644
index 0000000..71e0a27
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMAssignWithRandExec.java
@@ -0,0 +1,53 @@
+/**
+ * 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.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestAMAssignWithRandExec extends TestAssignmentManagerBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAMAssignWithRandExec.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestAMAssignWithRandExec.class);
+
+  @Test
+  public void testAssignWithRandExec() throws Exception {
+    TableName tableName = TableName.valueOf("testAssignWithRandExec");
+    RegionInfo 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("ROUND=" + i);
+      TransitRegionStateProcedure proc = createAssignProcedure(hri);
+      waitOnFuture(submitProcedure(proc));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMServerFailedOpen.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMServerFailedOpen.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMServerFailedOpen.java
new file mode 100644
index 0000000..b4689e5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAMServerFailedOpen.java
@@ -0,0 +1,134 @@
+/**
+ * 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.fail;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RetriesExhaustedException;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestAMServerFailedOpen extends TestAssignmentManagerBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAMServerFailedOpen.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestAMServerFailedOpen.class);
+
+  @Override
+  protected int getAssignMaxAttempts() {
+    // do not need to retry so many times as we will finally fail...
+    return 10;
+  }
+
+  @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 {
+    RegionInfo hri = createRegionInfo(tableName, 1);
+
+    // collect AM metrics before test
+    collectAssignmentManagerMetrics();
+
+    // Test Assign operation failure
+    rsDispatcher.setMockRsExecutor(executor);
+    try {
+      waitOnFuture(submitProcedure(createAssignProcedure(hri)));
+      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(createAssignProcedure(hri)));
+
+    // 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(createUnassignProcedure(hri)));
+     * assertEquals(assignSubmittedCount + 2, assignProcMetrics.getSubmittedCounter().getCount());
+     * assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
+     * assertEquals(unassignSubmittedCount + 1,
+     * unassignProcMetrics.getSubmittedCounter().getCount()); // TODO: We supposed to have 1 failed
+     * assign, 1 successful assign and a failed unassign // operation. But ProcV2 framework marks
+     * aborted unassign operation as success. Fix it! assertEquals(unassignFailedCount,
+     * unassignProcMetrics.getFailedCounter().getCount());
+     */
+  }
+
+  @Test
+  public void testIOExceptionOnAssignment() throws Exception {
+    // collect AM metrics before test
+    collectAssignmentManagerMetrics();
+
+    testFailedOpen(TableName.valueOf("testExceptionOnAssignment"),
+      new FaultyRsExecutor(new IOException("test fault")));
+
+    assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
+    assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
+  }
+
+  @Test
+  public void testDoNotRetryExceptionOnAssignment() throws Exception {
+    // collect AM metrics before test
+    collectAssignmentManagerMetrics();
+
+    testFailedOpen(TableName.valueOf("testDoNotRetryExceptionOnAssignment"),
+      new FaultyRsExecutor(new DoNotRetryIOException("test do not retry fault")));
+
+    assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
+    assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
+  }
+
+  private void testFailedOpen(final TableName tableName, final MockRSExecutor executor)
+      throws Exception {
+    final RegionInfo hri = createRegionInfo(tableName, 1);
+
+    // Test Assign operation failure
+    rsDispatcher.setMockRsExecutor(executor);
+    try {
+      waitOnFuture(submitProcedure(createAssignProcedure(hri)));
+      fail("unexpected assign completion");
+    } catch (RetriesExhaustedException e) {
+      // expected exception
+      LOG.info("REGION STATE " + am.getRegionStates().getRegionStateNode(hri));
+      LOG.info("expected exception from assign operation: " + e.getMessage(), e);
+      assertEquals(true, am.getRegionStates().getRegionState(hri).isFailedOpen());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 443bbab..94963a0 100644
--- 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
@@ -18,151 +18,43 @@
 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.io.InterruptedIOException;
-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.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.NotServingRegionException;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-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.MasterProcedureEnv;
-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.ProcedureMetrics;
 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.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.ipc.RemoteException;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Ignore;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-
-@Category({MasterTests.class, LargeTests.class})
-public class TestAssignmentManager {
+@Category({ MasterTests.class, LargeTests.class })
+public class TestAssignmentManager extends TestAssignmentManagerBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAssignmentManager.class);
+    HBaseClassTestRule.forClass(TestAssignmentManager.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentManager.class);
 
-  @Rule public TestName name = new TestName();
-  @Rule public final ExpectedException exception = ExpectedException.none();
-
-  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 ProcedureMetrics assignProcMetrics;
-  private ProcedureMetrics unassignProcMetrics;
-
-  private long assignSubmittedCount = 0;
-  private long assignFailedCount = 0;
-  private long unassignSubmittedCount = 0;
-  private long unassignFailedCount = 0;
-
-  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();
-    assignProcMetrics = am.getAssignmentManagerMetrics().getAssignProcMetrics();
-    unassignProcMetrics = am.getAssignmentManagerMetrics().getUnassignProcMetrics();
-    setUpMeta();
-  }
-
-  private void setUpMeta() throws Exception {
-    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
-    am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
-    am.wakeMetaLoadedEvent();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    master.stop("tearDown");
-    this.executor.shutdownNow();
-  }
-
-  @Test (expected=NullPointerException.class)
+  @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();
+    if (this.am.waitServerReportEvent(null, null)) {
+      throw new UnexpectedStateException();
+    }
   }
 
   @Test
@@ -173,61 +65,48 @@ public class TestAssignmentManager {
     testAssign(new GoodRsExecutor());
 
     assertEquals(assignSubmittedCount + NREGIONS,
-        assignProcMetrics.getSubmittedCounter().getCount());
+      assignProcMetrics.getSubmittedCounter().getCount());
     assertEquals(assignFailedCount, assignProcMetrics.getFailedCounter().getCount());
   }
 
   @Test
   public void testAssignAndCrashBeforeResponse() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
-    final RegionInfo hri = createRegionInfo(tableName, 1);
+    TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
+    RegionInfo hri = createRegionInfo(tableName, 1);
     rsDispatcher.setMockRsExecutor(new HangThenRSCrashExecutor());
-    AssignProcedure proc = am.createAssignProcedure(hri);
+    TransitRegionStateProcedure proc = createAssignProcedure(hri);
     waitOnFuture(submitProcedure(proc));
   }
 
   @Test
   public void testUnassignAndCrashBeforeResponse() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
-    final RegionInfo hri = createRegionInfo(tableName, 1);
+    TableName tableName = TableName.valueOf("testAssignAndCrashBeforeResponse");
+    RegionInfo hri = createRegionInfo(tableName, 1);
     rsDispatcher.setMockRsExecutor(new HangOnCloseThenRSCrashExecutor());
     for (int i = 0; i < HangOnCloseThenRSCrashExecutor.TYPES_OF_FAILURE; i++) {
-      AssignProcedure assign = am.createAssignProcedure(hri);
+      TransitRegionStateProcedure assign = createAssignProcedure(hri);
       waitOnFuture(submitProcedure(assign));
-      UnassignProcedure unassign = am.createUnassignProcedure(hri,
-          am.getRegionStates().getRegionServerOfRegion(hri), false);
+      TransitRegionStateProcedure unassign = createUnassignProcedure(hri);
       waitOnFuture(submitProcedure(unassign));
     }
   }
 
+  // Disabled for now. Since HBASE-18551, this mock is insufficient.
+  @Ignore
   @Test
-  public void testAssignWithRandExec() throws Exception {
-    final TableName tableName = TableName.valueOf("testAssignWithRandExec");
-    final RegionInfo 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("ROUND=" + i);
-      AssignProcedure proc = am.createAssignProcedure(hri);
-      waitOnFuture(submitProcedure(proc));
-    }
-  }
-
-  @Ignore @Test // Disabled for now. Since HBASE-18551, this mock is insufficient.
   public void testSocketTimeout() throws Exception {
-    final TableName tableName = TableName.valueOf(this.name.getMethodName());
-    final RegionInfo hri = createRegionInfo(tableName, 1);
+    TableName tableName = TableName.valueOf(this.name.getMethodName());
+    RegionInfo hri = createRegionInfo(tableName, 1);
 
     // collect AM metrics before test
     collectAssignmentManagerMetrics();
 
     rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 3));
-    waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
+    waitOnFuture(submitProcedure(createAssignProcedure(hri)));
 
     rsDispatcher.setMockRsExecutor(new SocketTimeoutRsExecutor(20, 1));
     // exception.expect(ServerCrashException.class);
-    waitOnFuture(submitProcedure(am.createUnassignProcedure(hri, null, false)));
+    waitOnFuture(submitProcedure(createUnassignProcedure(hri)));
 
     assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
     assertEquals(assignFailedCount, assignProcMetrics.getFailedCounter().getCount());
@@ -235,113 +114,26 @@ public class TestAssignmentManager {
     assertEquals(unassignFailedCount + 1, unassignProcMetrics.getFailedCounter().getCount());
   }
 
-  @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 RegionInfo hri = createRegionInfo(tableName, 1);
-
-    // collect AM metrics before test
-    collectAssignmentManagerMetrics();
-
-    // Test Assign operation failure
-    rsDispatcher.setMockRsExecutor(executor);
-    try {
-      waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
-      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)));
-
-    // 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)));
-
-    assertEquals(assignSubmittedCount + 2, assignProcMetrics.getSubmittedCounter().getCount());
-    assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
-    assertEquals(unassignSubmittedCount + 1, unassignProcMetrics.getSubmittedCounter().getCount());
-
-    // TODO: We supposed to have 1 failed assign, 1 successful assign and a failed unassign
-    // operation. But ProcV2 framework marks aborted unassign operation as success. Fix it!
-    assertEquals(unassignFailedCount, unassignProcMetrics.getFailedCounter().getCount());
-    */
-  }
-
-
-  @Test
-  public void testIOExceptionOnAssignment() throws Exception {
-    // collect AM metrics before test
-    collectAssignmentManagerMetrics();
-
-    testFailedOpen(TableName.valueOf("testExceptionOnAssignment"),
-      new FaultyRsExecutor(new IOException("test fault")));
-
-    assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
-    assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
-  }
-
-  @Test
-  public void testDoNotRetryExceptionOnAssignment() throws Exception {
-    // collect AM metrics before test
-    collectAssignmentManagerMetrics();
-
-    testFailedOpen(TableName.valueOf("testDoNotRetryExceptionOnAssignment"),
-      new FaultyRsExecutor(new DoNotRetryIOException("test do not retry fault")));
-
-    assertEquals(assignSubmittedCount + 1, assignProcMetrics.getSubmittedCounter().getCount());
-    assertEquals(assignFailedCount + 1, assignProcMetrics.getFailedCounter().getCount());
-  }
-
-  private void testFailedOpen(final TableName tableName,
-      final MockRSExecutor executor) throws Exception {
-    final RegionInfo hri = createRegionInfo(tableName, 1);
-
-    // Test Assign operation failure
-    rsDispatcher.setMockRsExecutor(executor);
-    try {
-      waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
-      fail("unexpected assign completion");
-    } catch (RetriesExhaustedException e) {
-      // expected exception
-      LOG.info("REGION STATE " + am.getRegionStates().getRegionStateNode(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 {
+  private void testAssign(MockRSExecutor executor, int nRegions) throws Exception {
     rsDispatcher.setMockRsExecutor(executor);
 
-    AssignProcedure[] assignments = new AssignProcedure[nregions];
+    TransitRegionStateProcedure[] assignments = new TransitRegionStateProcedure[nRegions];
 
     long st = System.currentTimeMillis();
     bulkSubmit(assignments);
 
     for (int i = 0; i < assignments.length; ++i) {
-      ProcedureTestingUtility.waitProcedure(
-        master.getMasterProcedureExecutor(), assignments[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));
+    LOG.info(String.format("[T] Assigning %dprocs in %s (%.2fproc/sec)", assignments.length,
+      StringUtils.humanTimeDiff(et - st), assignments.length / sec));
   }
 
   @Test
@@ -354,7 +146,7 @@ public class TestAssignmentManager {
 
     rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
 
-    final Future<byte[]> futureA = submitProcedure(am.createAssignProcedure(hri));
+    Future<byte[]> futureA = submitProcedure(createAssignProcedure(hri));
 
     // wait first assign
     waitOnFuture(futureA);
@@ -362,7 +154,7 @@ public class TestAssignmentManager {
     // 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));
+    Future<byte[]> futureB = submitProcedure(createAssignProcedure(hri));
     waitOnFuture(futureB);
     am.getRegionStates().isRegionInState(hri, State.OPEN);
     // TODO: What else can we do to ensure just a noop.
@@ -371,7 +163,6 @@ public class TestAssignmentManager {
     // better way?
     assertEquals(assignSubmittedCount + 2, assignProcMetrics.getSubmittedCounter().getCount());
     assertEquals(assignFailedCount, assignProcMetrics.getFailedCounter().getCount());
-
   }
 
   @Test
@@ -385,18 +176,16 @@ public class TestAssignmentManager {
     rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
 
     // assign the region first
-    waitOnFuture(submitProcedure(am.createAssignProcedure(hri)));
+    waitOnFuture(submitProcedure(createAssignProcedure(hri)));
 
-    final Future<byte[]> futureA = submitProcedure(am.createUnassignProcedure(hri, null, false));
+    final Future<byte[]> futureA = submitProcedure(createUnassignProcedure(hri));
 
     // 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));
+    final Future<byte[]> futureB = submitProcedure(createUnassignProcedure(hri));
     waitOnFuture(futureB);
     // Ensure we are still CLOSED.
     am.getRegionStates().isRegionInState(hri, State.CLOSED);
@@ -411,17 +200,17 @@ public class TestAssignmentManager {
   }
 
   /**
-   * It is possible that when AM send assign meta request to a RS successfully,
-   * but RS can not send back any response, which cause master startup hangs forever
+   * It is possible that when AM send assign meta request to a RS successfully, but RS can not send
+   * back any response, which cause master startup hangs forever
    */
   @Test
   public void testAssignMetaAndCrashBeforeResponse() throws Exception {
     tearDown();
     // See setUp(), start HBase until set up meta
-    UTIL = new HBaseTestingUtility();
+    util = new HBaseTestingUtility();
     this.executor = Executors.newSingleThreadScheduledExecutor();
-    setupConfiguration(UTIL.getConfiguration());
-    master = new MockMasterServices(UTIL.getConfiguration(), this.regionsToRegionServers);
+    setupConfiguration(util.getConfiguration());
+    master = new MockMasterServices(util.getConfiguration(), this.regionsToRegionServers);
     rsDispatcher = new MockRSProcedureDispatcher(master);
     master.start(NSERVERS, rsDispatcher);
     am = master.getAssignmentManager();
@@ -434,418 +223,4 @@ public class TestAssignmentManager {
     // set it back as default, see setUpMeta()
     am.wakeMetaLoadedEvent();
   }
-
-  private Future<byte[]> submitProcedure(final Procedure<MasterProcedureEnv> 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);
-      Exception ee = (Exception)e.getCause();
-      if (ee instanceof InterruptedIOException) {
-        for (Procedure<?> p: this.master.getMasterProcedureExecutor().getProcedures()) {
-          LOG.info(p.toStringDetails());
-        }
-      }
-      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) {
-    RegionInfo hri = createRegionInfo(tableName, regionId);
-    AssignProcedure proc = am.createAssignProcedure(hri);
-    master.getMasterProcedureExecutor().submitProcedure(proc);
-    return proc;
-  }
-
-  private RegionInfo createRegionInfo(final TableName tableName, final long regionId) {
-    return RegionInfoBuilder.newBuilder(tableName)
-        .setStartKey(Bytes.toBytes(regionId))
-        .setEndKey(Bytes.toBytes(regionId + 1))
-        .setSplit(false)
-        .setRegionId(0)
-        .build();
-  }
-
-  private void sendTransitionReport(final ServerName serverName,
-      final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.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 void doRestart(final ServerName serverName) {
-    try {
-      this.master.restartRegionServer(serverName);
-    } catch (IOException e) {
-      LOG.warn("Can not restart RS with new startcode");
-    }
-  }
-
-  private class NoopRsExecutor implements MockRSExecutor {
-    @Override
-    public ExecuteProceduresResponse sendRequest(ServerName server,
-        ExecuteProceduresRequest request) throws IOException {
-      if (request.getOpenRegionCount() > 0) {
-        for (OpenRegionRequest req : request.getOpenRegionList()) {
-          for (RegionOpenInfo openReq : req.getOpenInfoList()) {
-            execOpenRegion(server, openReq);
-          }
-        }
-      }
-      if (request.getCloseRegionCount() > 0) {
-        for (CloseRegionRequest req : request.getCloseRegionList()) {
-          execCloseRegion(server, req.getRegion().getValue().toByteArray());
-        }
-      }
-      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);
-      }
-      RegionInfo hri = ProtobufUtil.toRegionInfo(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 {
-      RegionInfo hri = am.getRegionInfo(regionName);
-      sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED);
-      return CloseRegionResponse.newBuilder().setClosed(true).build();
-    }
-  }
-
-  private static class ServerNotYetRunningRsExecutor implements MockRSExecutor {
-    @Override
-    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;
-    }
-
-    @Override
-    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;
-    }
-
-    @Override
-    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;
-    }
-  }
-
-  /**
-   * 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.).
-   * Different with HangThenRSCrashExecutor,  HangThenRSCrashExecutor will create
-   * ServerCrashProcedure to handle the server crash. However, this HangThenRSRestartExecutor
-   * will restart RS directly, situation for RS crashed when SCP is not enabled.
-   */
-  private class HangThenRSRestartExecutor 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("Restarting RS of " + server);
-          doRestart(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:
-          executor.schedule(new Runnable() {
-            @Override
-            public void run() {
-              LOG.info("Sending in CRASH of " + server);
-              doCrash(server);
-            }
-          }, 1, TimeUnit.SECONDS);
-          throw new RegionServerAbortedException("Fake!");
-        case 2:
-          executor.schedule(new Runnable() {
-            @Override
-            public void run() {
-              LOG.info("Sending in CRASH of " + server);
-              doCrash(server);
-            }
-          }, 1, TimeUnit.SECONDS);
-          throw new RegionServerStoppedException("Fake!");
-        case 3: throw new ServerNotRunningYetException("Fake!");
-        case 4:
-          LOG.info("Returned null from serverName={}; means STUCK...TODO timeout", server);
-          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();
-
-    @Override
-    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");
-        default:
-          // fall out
-      }
-      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;
-        default:
-          // fall out
-      }
-      // 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) {
-        RegionInfo hri = am.getRegionInfo(regionName);
-        sendTransitionReport(server, ProtobufUtil.toRegionInfo(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> remoteProcedures) {
-      submitTask(new MockRemoteCall(serverName, remoteProcedures));
-    }
-
-    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 void collectAssignmentManagerMetrics() {
-    assignSubmittedCount = assignProcMetrics.getSubmittedCounter().getCount();
-    assignFailedCount = assignProcMetrics.getFailedCounter().getCount();
-    unassignSubmittedCount = unassignProcMetrics.getSubmittedCounter().getCount();
-    unassignFailedCount = unassignProcMetrics.getFailedCounter().getCount();
-  }
 }


[4/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
new file mode 100644
index 0000000..e853b9b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
@@ -0,0 +1,569 @@
+/**
+ * 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 edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+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.procedure.AbstractStateMachineRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionStateTransitionStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
+/**
+ * The procedure to deal with the state transition of a region. A region with a TRSP in place is
+ * called RIT, i.e, RegionInTransition.
+ * <p/>
+ * It can be used to assign/unassign/reopen/move a region, and for
+ * {@link #unassign(MasterProcedureEnv, RegionInfo)} and
+ * {@link #reopen(MasterProcedureEnv, RegionInfo)}, you do not need to specify a target server, and
+ * for {@link #assign(MasterProcedureEnv, RegionInfo, ServerName)} and
+ * {@link #move(MasterProcedureEnv, RegionInfo, ServerName)}, if you want to you can provide a
+ * target server. And for {@link #move(MasterProcedureEnv, RegionInfo, ServerName)}, if you do not
+ * specify a targetServer, we will select one randomly.
+ * <p/>
+ * <p/>
+ * The typical state transition for assigning a region is:
+ *
+ * <pre>
+ * GET_ASSIGN_CANDIDATE ------> OPEN -----> CONFIRM_OPENED
+ * </pre>
+ *
+ * Notice that, if there are failures we may go back to the {@code GET_ASSIGN_CANDIDATE} state to
+ * try again.
+ * <p/>
+ * The typical state transition for unassigning a region is:
+ *
+ * <pre>
+ * CLOSE -----> CONFIRM_CLOSED
+ * </pre>
+ *
+ * Here things go a bit different, if there are failures, especially that if there is a server
+ * crash, we will go to the {@code GET_ASSIGN_CANDIDATE} state to bring the region online first, and
+ * then go through the normal way to unassign it.
+ * <p/>
+ * The typical state transition for reopening/moving a region is:
+ *
+ * <pre>
+ * CLOSE -----> CONFIRM_CLOSED -----> GET_ASSIGN_CANDIDATE ------> OPEN -----> CONFIRM_OPENED
+ * </pre>
+ *
+ * The retry logic is the same with the above assign/unassign.
+ * <p/>
+ * Notice that, although we allow specify a target server, it just acts as a candidate, we do not
+ * guarantee that the region will finally be on the target server. If this is important for you, you
+ * should check whether the region is on the target server after the procedure is finished.
+ * <p/>
+ * When you want to schedule a TRSP, please check whether there is still one for this region, and
+ * the check should be under the RegionStateNode lock. We will remove the TRSP from a
+ * RegionStateNode when we are done, see the code in {@code reportTransition} method below. There
+ * could be at most one TRSP for a give region.
+ */
+@InterfaceAudience.Private
+public class TransitRegionStateProcedure
+    extends AbstractStateMachineRegionProcedure<RegionStateTransitionState> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TransitRegionStateProcedure.class);
+
+  private RegionStateTransitionState initialState;
+
+  private RegionStateTransitionState lastState;
+
+  // the candidate where we want to assign the region to.
+  private ServerName assignCandidate;
+
+  private boolean forceNewPlan;
+
+  private int attempt;
+
+  public TransitRegionStateProcedure() {
+  }
+
+  private TransitRegionStateProcedure(MasterProcedureEnv env, RegionInfo hri,
+      ServerName assignCandidate, boolean forceNewPlan, RegionStateTransitionState initialState,
+      RegionStateTransitionState lastState) {
+    super(env, hri);
+    this.assignCandidate = assignCandidate;
+    this.forceNewPlan = forceNewPlan;
+    this.initialState = initialState;
+    this.lastState = lastState;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    // TODO: maybe we should make another type here, REGION_TRANSITION?
+    return TableOperationType.REGION_EDIT;
+  }
+
+  @Override
+  protected boolean waitInitialized(MasterProcedureEnv env) {
+    if (TableName.isMetaTableName(getTableName())) {
+      return false;
+    }
+    // First we need meta to be loaded, and second, if meta is not online then we will likely to
+    // fail when updating meta so we wait until it is assigned.
+    AssignmentManager am = env.getAssignmentManager();
+    return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, getRegion());
+  }
+
+  private void queueAssign(MasterProcedureEnv env, RegionStateNode regionNode)
+      throws ProcedureSuspendedException {
+    // Here the assumption is that, the region must be in CLOSED state, so the region location
+    // will be null. And if we fail to open the region and retry here, the forceNewPlan will be
+    // true, and also we will set the region location to null.
+    boolean retain = false;
+    if (!forceNewPlan) {
+      if (assignCandidate != null) {
+        retain = assignCandidate.equals(regionNode.getLastHost());
+        regionNode.setRegionLocation(assignCandidate);
+      } else if (regionNode.getLastHost() != null) {
+        retain = true;
+        LOG.info("Setting lastHost as the region location {}", regionNode.getLastHost());
+        regionNode.setRegionLocation(regionNode.getLastHost());
+      }
+    }
+    LOG.info("Starting {}; {}; forceNewPlan={}, retain={}", this, regionNode.toShortString(),
+      forceNewPlan, retain);
+    env.getAssignmentManager().queueAssign(regionNode);
+    setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_OPEN);
+    if (regionNode.getProcedureEvent().suspendIfNotReady(this)) {
+      throw new ProcedureSuspendedException();
+    }
+  }
+
+  private void openRegion(MasterProcedureEnv env, RegionStateNode regionNode) throws IOException {
+    ServerName loc = regionNode.getRegionLocation();
+    if (loc == null) {
+      LOG.warn("No location specified for {}, jump back to state {} to get one", getRegion(),
+        RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+      return;
+    }
+    env.getAssignmentManager().regionOpening(regionNode);
+    addChildProcedure(new OpenRegionProcedure(getRegion(), loc));
+    setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED);
+  }
+
+  private Flow confirmOpened(MasterProcedureEnv env, RegionStateNode regionNode)
+      throws IOException {
+    // notice that, for normal case, if we successfully opened a region, we will not arrive here, as
+    // in reportTransition we will call unsetProcedure, and in executeFromState we will return
+    // directly. But if the master is crashed before we finish the procedure, then next time we will
+    // arrive here. So we still need to add code for normal cases.
+    if (regionNode.isInState(State.OPEN)) {
+      attempt = 0;
+      if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED) {
+        // we are the last state, finish
+        regionNode.unsetProcedure(this);
+        return Flow.NO_MORE_STATE;
+      }
+      // It is possible that we arrive here but confirm opened is not the last state, for example,
+      // when merging or splitting a region, we unassign the region from a RS and the RS is crashed,
+      // then there will be recovered edits for this region, we'd better make the region online
+      // again and then unassign it, otherwise we have to fail the merge/split procedure as we may
+      // loss data.
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE);
+      return Flow.HAS_MORE_STATE;
+    }
+
+    if (incrementAndCheckMaxAttempts(env, regionNode)) {
+      env.getAssignmentManager().regionFailedOpen(regionNode, true);
+      setFailure(getClass().getSimpleName(), new RetriesExhaustedException(
+        "Max attempts " + env.getAssignmentManager().getAssignMaxAttempts() + " exceeded"));
+      regionNode.unsetProcedure(this);
+      return Flow.NO_MORE_STATE;
+    }
+    env.getAssignmentManager().regionFailedOpen(regionNode, false);
+    // we failed to assign the region, force a new plan
+    forceNewPlan = true;
+    regionNode.setRegionLocation(null);
+    setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+    // Here we do not throw exception because we want to the region to be online ASAP
+    return Flow.HAS_MORE_STATE;
+  }
+
+  private void closeRegion(MasterProcedureEnv env, RegionStateNode regionNode) throws IOException {
+    if (regionNode.isInState(State.OPEN, State.CLOSING, State.MERGING, State.SPLITTING)) {
+      // this is the normal case
+      env.getAssignmentManager().regionClosing(regionNode);
+      addChildProcedure(
+        new CloseRegionProcedure(getRegion(), regionNode.getRegionLocation(), assignCandidate));
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED);
+    } else {
+      forceNewPlan = true;
+      regionNode.setRegionLocation(null);
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+    }
+  }
+
+  private Flow confirmClosed(MasterProcedureEnv env, RegionStateNode regionNode)
+      throws IOException {
+    // notice that, for normal case, if we successfully opened a region, we will not arrive here, as
+    // in reportTransition we will call unsetProcedure, and in executeFromState we will return
+    // directly. But if the master is crashed before we finish the procedure, then next time we will
+    // arrive here. So we still need to add code for normal cases.
+    if (regionNode.isInState(State.CLOSED)) {
+      attempt = 0;
+      if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
+        // we are the last state, finish
+        regionNode.unsetProcedure(this);
+        return Flow.NO_MORE_STATE;
+      }
+      // This means we need to open the region again, should be a move or reopen
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+      return Flow.HAS_MORE_STATE;
+    }
+    if (regionNode.isInState(State.CLOSING)) {
+      // This is possible, think the target RS crashes and restarts immediately, the close region
+      // operation will return a NotServingRegionException soon, we can only recover after SCP takes
+      // care of this RS. So here we throw an IOException to let upper layer to retry with backoff.
+      setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE);
+      throw new HBaseIOException("Failed to close region");
+    }
+    // abnormally closed, need to reopen it, no matter what is the last state, see the comment in
+    // confirmOpened for more details that why we need to reopen the region first even if we just
+    // want to close it.
+    // The only exception is for non-default replica, where we do not need to deal with recovered
+    // edits. Notice that the region will remain in ABNORMALLY_CLOSED state, the upper layer need to
+    // deal with this state. For non-default replica, this is usually the same with CLOSED.
+    assert regionNode.isInState(State.ABNORMALLY_CLOSED);
+    if (!RegionReplicaUtil.isDefaultReplica(getRegion()) &&
+      lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
+      regionNode.unsetProcedure(this);
+      return Flow.NO_MORE_STATE;
+    }
+    attempt = 0;
+    setNextState(RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE);
+    return Flow.HAS_MORE_STATE;
+  }
+
+  // Override to lock RegionStateNode
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected Procedure[] execute(MasterProcedureEnv env)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    RegionStateNode regionNode =
+      env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(getRegion());
+    regionNode.lock();
+    try {
+      return super.execute(env);
+    } finally {
+      regionNode.unlock();
+    }
+  }
+
+  private RegionStateNode getRegionStateNode(MasterProcedureEnv env) {
+    return env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(getRegion());
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, RegionStateTransitionState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    RegionStateNode regionNode = getRegionStateNode(env);
+    if (regionNode.getProcedure() != this) {
+      // This is possible, and is the normal case, as we will call unsetProcedure in
+      // reportTransition, this means we have already done
+      // This is because that, when we mark the region as OPENED or CLOSED, then all the works
+      // should have already been done, and logically we could have another TRSP scheduled for this
+      // region immediately(think of a RS crash at the point...).
+      return Flow.NO_MORE_STATE;
+    }
+    try {
+      switch (state) {
+        case REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE:
+          queueAssign(env, regionNode);
+          return Flow.HAS_MORE_STATE;
+        case REGION_STATE_TRANSITION_OPEN:
+          openRegion(env, regionNode);
+          return Flow.HAS_MORE_STATE;
+        case REGION_STATE_TRANSITION_CONFIRM_OPENED:
+          return confirmOpened(env, regionNode);
+        case REGION_STATE_TRANSITION_CLOSE:
+          closeRegion(env, regionNode);
+          return Flow.HAS_MORE_STATE;
+        case REGION_STATE_TRANSITION_CONFIRM_CLOSED:
+          return confirmClosed(env, regionNode);
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      long backoff = getBackoffTime(this.attempt++);
+      LOG.warn(
+        "Failed transition, suspend {}secs {}; {}; waiting on rectified condition fixed " +
+          "by other Procedure or operator intervention",
+        backoff / 1000, this, regionNode.toShortString(), e);
+      regionNode.getProcedureEvent().suspend();
+      if (regionNode.getProcedureEvent().suspendIfNotReady(this)) {
+        setTimeout(Math.toIntExact(backoff));
+        setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+        throw new ProcedureSuspendedException();
+      }
+      return Flow.HAS_MORE_STATE;
+    }
+  }
+
+  /**
+   * At end of timeout, wake ourselves up so we run again.
+   */
+  @Override
+  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+    setState(ProcedureProtos.ProcedureState.RUNNABLE);
+    getRegionStateNode(env).getProcedureEvent().wake(env.getProcedureScheduler());
+    return false; // 'false' means that this procedure handled the timeout
+  }
+
+  private void reportTransitionOpened(MasterProcedureEnv env, RegionStateNode regionNode,
+      ServerName serverName, TransitionCode code, long openSeqNum) throws IOException {
+    switch (code) {
+      case OPENED:
+        if (openSeqNum < 0) {
+          throw new UnexpectedStateException("Received report unexpected " + code +
+            " transition openSeqNum=" + openSeqNum + ", " + regionNode);
+        }
+        if (openSeqNum <= regionNode.getOpenSeqNum()) {
+          if (openSeqNum != 0) {
+            LOG.warn("Skip update of openSeqNum for {} with {} because the currentSeqNum={}",
+              regionNode, openSeqNum, regionNode.getOpenSeqNum());
+          }
+        } else {
+          regionNode.setOpenSeqNum(openSeqNum);
+        }
+        env.getAssignmentManager().regionOpened(regionNode);
+        if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED) {
+          // we are done
+          regionNode.unsetProcedure(this);
+        }
+        regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+        break;
+      case FAILED_OPEN:
+        // just wake up the procedure and see if we can retry
+        regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+        break;
+      default:
+        throw new UnexpectedStateException(
+          "Received report unexpected " + code + " transition openSeqNum=" + openSeqNum + ", " +
+            regionNode.toShortString() + ", " + this + ", expected OPENED or FAILED_OPEN.");
+    }
+  }
+
+  // we do not need seqId for closing a region
+  private void reportTransitionClosed(MasterProcedureEnv env, RegionStateNode regionNode,
+      ServerName serverName, TransitionCode code) throws IOException {
+    switch (code) {
+      case CLOSED:
+        env.getAssignmentManager().regionClosed(regionNode, true);
+        if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
+          // we are done
+          regionNode.unsetProcedure(this);
+        }
+        regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+        break;
+      default:
+        throw new UnexpectedStateException("Received report unexpected " + code + " transition, " +
+          regionNode.toShortString() + ", " + this + ", expected CLOSED.");
+    }
+  }
+
+  // Should be called with RegionStateNode locked
+  public void reportTransition(MasterProcedureEnv env, RegionStateNode regionNode,
+      ServerName serverName, TransitionCode code, long seqId) throws IOException {
+    switch (getCurrentState()) {
+      case REGION_STATE_TRANSITION_CONFIRM_OPENED:
+        reportTransitionOpened(env, regionNode, serverName, code, seqId);
+        break;
+      case REGION_STATE_TRANSITION_CONFIRM_CLOSED:
+        reportTransitionClosed(env, regionNode, serverName, code);
+        break;
+      default:
+        LOG.warn("{} received unexpected report transition call from {}, code={}, seqId={}", this,
+          serverName, code, seqId);
+    }
+  }
+
+  // Should be called with RegionStateNode locked
+  public void serverCrashed(MasterProcedureEnv env, RegionStateNode regionNode,
+      ServerName serverName) throws IOException {
+    // Notice that, in this method, we do not change the procedure state, instead, we update the
+    // region state in hbase:meta. This is because that, the procedure state change will not be
+    // persisted until the region is woken up and finish one step, if we crash before that then the
+    // information will be lost. So here we will update the region state in hbase:meta, and when the
+    // procedure is woken up, it will process the error and jump to the correct procedure state.
+    RegionStateTransitionState currentState = getCurrentState();
+    switch (currentState) {
+      case REGION_STATE_TRANSITION_CLOSE:
+      case REGION_STATE_TRANSITION_CONFIRM_CLOSED:
+      case REGION_STATE_TRANSITION_CONFIRM_OPENED:
+        // for these 3 states, the region may still be online on the crashed server
+        if (serverName.equals(regionNode.getRegionLocation())) {
+          env.getAssignmentManager().regionClosed(regionNode, false);
+          if (currentState != RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE) {
+            regionNode.getProcedureEvent().wake(env.getProcedureScheduler());
+          }
+        }
+        break;
+      default:
+        // If the procedure is in other 2 states, then actually we should not arrive here, as we
+        // know that the region is not online on any server, so we need to do nothing... But anyway
+        // let's add a log here
+        LOG.warn("{} received unexpected server crash call for region {} from {}", this, regionNode,
+          serverName);
+
+    }
+  }
+
+  private long getBackoffTime(int attempts) {
+    long backoffTime = (long) (1000 * Math.pow(2, attempts));
+    long maxBackoffTime = 60 * 60 * 1000; // An hour. Hard-coded for for now.
+    return backoffTime < maxBackoffTime ? backoffTime : maxBackoffTime;
+  }
+
+  private boolean incrementAndCheckMaxAttempts(MasterProcedureEnv env, RegionStateNode regionNode) {
+    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
+  protected void rollbackState(MasterProcedureEnv env, RegionStateTransitionState state)
+      throws IOException, InterruptedException {
+    // no rollback
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected RegionStateTransitionState getState(int stateId) {
+    return RegionStateTransitionState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(RegionStateTransitionState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected RegionStateTransitionState getInitialState() {
+    return initialState;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    RegionStateTransitionStateData.Builder builder = RegionStateTransitionStateData.newBuilder()
+      .setInitialState(initialState).setLastState(lastState).setForceNewPlan(forceNewPlan);
+    if (assignCandidate != null) {
+      builder.setAssignCandidate(ProtobufUtil.toServerName(assignCandidate));
+    }
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    RegionStateTransitionStateData data =
+      serializer.deserialize(RegionStateTransitionStateData.class);
+    initialState = data.getInitialState();
+    lastState = data.getLastState();
+    forceNewPlan = data.getForceNewPlan();
+    if (data.hasAssignCandidate()) {
+      assignCandidate = ProtobufUtil.toServerName(data.getAssignCandidate());
+    }
+  }
+
+  @Override
+  protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
+    // TODO: need to reimplement the metrics system for assign/unassign
+    if (initialState == RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE) {
+      return env.getAssignmentManager().getAssignmentManagerMetrics().getAssignProcMetrics();
+    } else {
+      return env.getAssignmentManager().getAssignmentManagerMetrics().getUnassignProcMetrics();
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    super.toStringClassDetails(sb);
+    if (initialState == RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE) {
+      sb.append(", ASSIGN");
+    } else if (lastState == RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED) {
+      sb.append(", UNASSIGN");
+    } else {
+      sb.append(", REOPEN/MOVE");
+    }
+  }
+
+  private static TransitRegionStateProcedure setOwner(MasterProcedureEnv env,
+      TransitRegionStateProcedure proc) {
+    proc.setOwner(env.getRequestUser().getShortName());
+    return proc;
+  }
+
+  // Be careful that, when you call these 4 methods below, you need to manually attach the returned
+  // procedure with the RegionStateNode, otherwise the procedure will quit immediately without doing
+  // anything. See the comment in executeFromState to find out why we need this assumption.
+  public static TransitRegionStateProcedure assign(MasterProcedureEnv env, RegionInfo region,
+      @Nullable ServerName targetServer) {
+    return setOwner(env,
+      new TransitRegionStateProcedure(env, region, targetServer, false,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED));
+  }
+
+  public static TransitRegionStateProcedure unassign(MasterProcedureEnv env, RegionInfo region) {
+    return setOwner(env,
+      new TransitRegionStateProcedure(env, region, null, false,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_CLOSED));
+  }
+
+  public static TransitRegionStateProcedure reopen(MasterProcedureEnv env, RegionInfo region) {
+    return setOwner(env,
+      new TransitRegionStateProcedure(env, region, null, false,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED));
+  }
+
+  public static TransitRegionStateProcedure move(MasterProcedureEnv env, RegionInfo region,
+      @Nullable ServerName targetServer) {
+    return setOwner(env,
+      new TransitRegionStateProcedure(env, region, targetServer, targetServer == null,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CLOSE,
+        RegionStateTransitionState.REGION_STATE_TRANSITION_CONFIRM_OPENED));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 4f58a0f..def8fd5 100644
--- 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
@@ -20,105 +20,38 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
-import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
-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.RegionCloseOperation;
-import org.apache.hadoop.hbase.master.procedure.ServerCrashException;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
-import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 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;
 
 /**
- * Procedure that describes the unassignment of a single region.
- * There can only be one RegionTransitionProcedure -- i.e. an assign or an unassign -- per region
- * running at a 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" (suspend).
- * 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 this procedure.
- * The procedure will wakeup and finish the unassign by publishing its new state on meta.
- * <p>If we are unable to contact the remote regionserver whether because of ConnectException
- * or socket timeout, we will call expire on the server we were trying to contact. We will remain
- * in suspended state waiting for a wake up from the ServerCrashProcedure that is processing the
- * failed server. The basic idea is that if we notice a crashed server, then we have a
- * responsibility; i.e. we should not let go of the region until we are sure the server that was
- * hosting has had its crash processed. If we let go of the region before then, an assign might
- * run before the logs have been split which would make for data loss.
- *
- * <p>TODO: Rather than this tricky coordination between SCP and this Procedure, instead, work on
- * returning a SCP as our subprocedure; probably needs work on the framework to do this,
- * especially if the SCP already created.
+ * Leave here only for checking if we can successfully start the master.
+ * @deprecated Do not use any more.
+ * @see TransitRegionStateProcedure
  */
+@Deprecated
 @InterfaceAudience.Private
 public class UnassignProcedure extends RegionTransitionProcedure {
-  private static final Logger LOG = LoggerFactory.getLogger(UnassignProcedure.class);
 
-  /**
-   * Where to send the unassign RPC.
-   */
   protected volatile ServerName hostingServer;
-  /**
-   * The Server we will subsequently assign the region too (can be null).
-   */
+
   protected volatile ServerName destinationServer;
 
-  // TODO: should this be in a reassign procedure?
-  //       ...and keep unassign for 'disable' case?
   private boolean force;
 
-  /**
-   * Whether deleting the region from in-memory states after unassigning the region.
-   */
   private boolean removeAfterUnassigning;
 
   public UnassignProcedure() {
-    // Required by the Procedure framework to create the procedure on replay
-    super();
-  }
-
-  public UnassignProcedure(final RegionInfo regionInfo, final ServerName hostingServer,
-      final boolean force, final boolean removeAfterUnassigning) {
-    this(regionInfo, hostingServer, null, force, removeAfterUnassigning);
-  }
-
-  public UnassignProcedure(final RegionInfo regionInfo,
-      final ServerName hostingServer, final ServerName destinationServer, final boolean force) {
-    this(regionInfo, hostingServer, destinationServer, force, false);
-  }
-
-  public UnassignProcedure(final RegionInfo regionInfo, final ServerName hostingServer,
-      final ServerName destinationServer, final boolean force,
-      final boolean removeAfterUnassigning) {
-    super(regionInfo);
-    this.hostingServer = hostingServer;
-    this.destinationServer = destinationServer;
-    this.force = force;
-    this.removeAfterUnassigning = removeAfterUnassigning;
-
-    // we don't need REGION_TRANSITION_QUEUE, we jump directly to sending the request
-    setTransitionState(RegionTransitionState.REGION_TRANSITION_DISPATCH);
   }
 
   @Override
@@ -138,10 +71,9 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
-    UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
-        .setTransitionState(getTransitionState())
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    UnassignRegionStateData.Builder state =
+      UnassignRegionStateData.newBuilder().setTransitionState(getTransitionState())
         .setHostingServer(ProtobufUtil.toServerName(this.hostingServer))
         .setRegionInfo(ProtobufUtil.toRegionInfo(getRegionInfo()));
     if (this.destinationServer != null) {
@@ -160,10 +92,8 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
-    final UnassignRegionStateData state =
-        serializer.deserialize(UnassignRegionStateData.class);
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    final UnassignRegionStateData state = serializer.deserialize(UnassignRegionStateData.class);
     setTransitionState(state.getTransitionState());
     setRegionInfo(ProtobufUtil.toRegionInfo(state.getRegionInfo()));
     this.hostingServer = ProtobufUtil.toServerName(state.getHostingServer());
@@ -178,7 +108,8 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  protected boolean startTransition(final MasterProcedureEnv env, final RegionStateNode regionNode) {
+  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();
@@ -186,53 +117,18 @@ public class UnassignProcedure extends RegionTransitionProcedure {
 
   @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 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 to the server dispatch queue.
-    if (!addToRemoteDispatcher(env, regionNode.getRegionLocation())) {
-      // If addToRemoteDispatcher fails, it calls the callback #remoteCallFailed.
-    }
-
-    // Return true to keep the procedure running.
+      throws IOException {
     return true;
   }
 
   @Override
   protected void finishTransition(final MasterProcedureEnv env, final RegionStateNode regionNode)
       throws IOException {
-    AssignmentManager am = env.getAssignmentManager();
-    RegionInfo regionInfo = getRegionInfo();
-
-    if (!removeAfterUnassigning) {
-      am.markRegionAsClosed(regionNode);
-    } else {
-      // Remove from in-memory states
-      am.getRegionStates().deleteRegion(regionInfo);
-      env.getMasterServices().getServerManager().removeRegion(regionInfo);
-      FavoredNodesManager fnm = env.getMasterServices().getFavoredNodesManager();
-      if (fnm != null) {
-        fnm.deleteFavoredNodesForRegions(Lists.newArrayList(regionInfo));
-      }
-    }
   }
 
   @Override
-  public RemoteOperation remoteCallBuild(final MasterProcedureEnv env, final ServerName serverName) {
+  public RemoteOperation remoteCallBuild(final MasterProcedureEnv env,
+      final ServerName serverName) {
     assert serverName.equals(getRegionState(env).getRegionLocation());
     return new RegionCloseOperation(this, getRegionInfo(), this.destinationServer);
   }
@@ -240,48 +136,6 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   @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()));
-    }
-  }
-
-  /**
-   * Our remote call failed but there are a few states where it is safe to proceed with the
-   * unassign; e.g. if a server crash and it has had all of its WALs processed, then we can allow
-   * this unassign to go to completion.
-   * @return True if it is safe to proceed with the unassign.
-   */
-  private boolean isSafeToProceed(final MasterProcedureEnv env, final RegionStateNode regionNode,
-    final IOException exception) {
-    if (exception instanceof ServerCrashException) {
-      // This exception comes from ServerCrashProcedure AFTER log splitting. Its a signaling
-      // exception. SCP found this region as a RIT during its processing of the crash.  Its call
-      // into here says it is ok to let this procedure go complete.
-      return true;
-    }
-    if (exception instanceof NotServingRegionException) {
-      LOG.warn("IS OK? ANY LOGS TO REPLAY; ACTING AS THOUGH ALL GOOD {}", regionNode, exception);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Set it up so when procedure is unsuspended, we'll move to the procedure finish.
-   */
-  protected void proceed(final MasterProcedureEnv env, final RegionStateNode regionNode) {
-    try {
-      reportTransition(env, regionNode, TransitionCode.CLOSED, HConstants.NO_SEQNUM);
-    } catch (UnexpectedStateException e) {
-      // Should never happen.
-      throw new RuntimeException(e);
-    }
   }
 
   /**
@@ -290,61 +144,6 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   @Override
   protected boolean remoteCallFailed(final MasterProcedureEnv env, final RegionStateNode regionNode,
       final IOException exception) {
-    // Be careful reading the below; we do returns in middle of the method a few times.
-    if (isSafeToProceed(env, regionNode, exception)) {
-      proceed(env, regionNode);
-    } else if (exception instanceof RegionServerAbortedException ||
-        exception instanceof RegionServerStoppedException) {
-      // RS is aborting/stopping, we cannot offline the region since the region may need to do WAL
-      // recovery. Until we see the RS expiration, stay suspended; return false.
-      LOG.info("Ignoring; waiting on ServerCrashProcedure", exception);
-      return false;
-    } else if (exception instanceof ServerNotRunningYetException) {
-      // This should not happen. If it does, procedure will be woken-up and we'll retry.
-      // TODO: Needs a pause and backoff?
-      LOG.info("Retry", exception);
-    } else {
-      // We failed to RPC this server. Set it as expired.
-      ServerName serverName = regionNode.getRegionLocation();
-      LOG.warn("Expiring {}, {} {}; exception={}", serverName, this, regionNode.toShortString(),
-          exception.getClass().getSimpleName());
-      if (!env.getMasterServices().getServerManager().expireServer(serverName)) {
-        // Failed to queue an expire. Lots of possible reasons including it may be already expired.
-        // In ServerCrashProcedure and RecoverMetaProcedure, there is a handleRIT stage where we
-        // will iterator over all the RIT procedures for the related regions of a crashed RS and
-        // fail them with ServerCrashException. You can see the isSafeToProceed method above for
-        // more details.
-        // This can work for most cases, but since we do not hold the region lock in handleRIT,
-        // there could be race that we arrive here after the handleRIT stage of the SCP. So here we
-        // need to check whether it is safe to quit.
-        // Notice that, the first assumption is that we can only quit after the log splitting is
-        // done, as MRP can schedule an AssignProcedure right after us, and if the log splitting has
-        // not been done then there will be data loss. And in SCP, we will change the state from
-        // SPLITTING to OFFLINE(or SPLITTING_META_DONE for meta log processing) after finishing the
-        // log splitting, and then calling handleRIT, so checking the state here can be a safe
-        // fence. If the state is not OFFLINE(or SPLITTING_META_DONE), then we can just leave this
-        // procedure in suspended state as we can make sure that the handleRIT has not been executed
-        // yet and it will wake us up later. And if the state is OFFLINE(or SPLITTING_META_DONE), we
-        // can safely quit since there will be no data loss. There could be duplicated
-        // AssignProcedures for the same region but it is OK as we will do a check at the beginning
-        // of AssignProcedure to prevent double assign. And there we have region lock so there will
-        // be no race.
-        if (env.getAssignmentManager().isLogSplittingDone(serverName, isMeta())) {
-          // Its ok to proceed with this unassign.
-          LOG.info("{} is dead and processed; moving procedure to finished state; {}", serverName,
-            this);
-          proceed(env, regionNode);
-          // Return true; wake up the procedure so we can act on proceed.
-          return true;
-        }
-      }
-      // Return false so this procedure stays in suspended state. It will be woken up by the
-      // ServerCrashProcedure that was scheduled when we called #expireServer above. SCP calls
-      // #handleRIT which will call this method only the exception will be a ServerCrashException
-      // this time around (See above).
-      // TODO: Add a SCP as a new subprocedure that we now come to depend on.
-      return false;
-    }
     return true;
   }
 
@@ -355,11 +154,6 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   @Override
-  public ServerName getServer(final MasterProcedureEnv env) {
-    return this.hostingServer;
-  }
-
-  @Override
   protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
     return env.getAssignmentManager().getAssignmentManagerMetrics().getUnassignProcMetrics();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 ff04b21..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
+++ /dev/null
@@ -1,72 +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.ServerName;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hbase.thirdparty.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 RegionInfo hri)
-  throws IOException {
-    return getRegionInfoResponse(env, regionLocation, hri, false);
-  }
-
-  static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
-      final ServerName regionLocation, final RegionInfo hri, boolean includeBestSplitRow)
-  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 = null;
-    if (includeBestSplitRow) {
-      request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName(), false, true);
-    } else {
-      request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName());
-    }
-    try {
-      return admin.getRegionInfo(controller, request);
-    } catch (ServiceException e) {
-      throw ProtobufUtil.handleRemoteException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 3b5e3b5..cf4818c 100644
--- 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
@@ -40,13 +40,12 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
     extends AbstractStateMachineTableProcedure<TState> {
   private RegionInfo hri;
 
-  public AbstractStateMachineRegionProcedure(final MasterProcedureEnv env,
-      final RegionInfo hri) {
+  protected AbstractStateMachineRegionProcedure(MasterProcedureEnv env, RegionInfo hri) {
     super(env);
     this.hri = hri;
   }
 
-  public AbstractStateMachineRegionProcedure() {
+  protected AbstractStateMachineRegionProcedure() {
     // Required by the Procedure framework to create the procedure on replay
     super();
   }
@@ -54,7 +53,7 @@ public abstract class AbstractStateMachineRegionProcedure<TState>
   /**
    * @return The RegionInfo of the region we are operating on.
    */
-  protected RegionInfo getRegion() {
+  public RegionInfo getRegion() {
     return this.hri;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 4c77f6b..ca4e9d6 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
@@ -28,12 +28,10 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionOfflineException;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.TableStateManager;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -180,26 +178,13 @@ public abstract class AbstractStateMachineTableProcedure<TState>
   /**
    * Check region is online.
    */
-  protected static void checkOnline(MasterProcedureEnv env, final RegionInfo ri)
+  protected static void checkOnline(MasterProcedureEnv env, RegionInfo ri)
       throws DoNotRetryRegionException {
-    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
-    RegionState rs = regionStates.getRegionState(ri);
-    if (rs == null) {
+    RegionStateNode regionNode =
+      env.getAssignmentManager().getRegionStates().getRegionStateNode(ri);
+    if (regionNode == null) {
       throw new UnknownRegionException("No RegionState found for " + ri.getEncodedName());
     }
-    if (!rs.isOpened()) {
-      throw new DoNotRetryRegionException(ri.getEncodedName() + " is not OPEN; regionState=" + rs);
-    }
-    if (ri.isSplitParent()) {
-      throw new DoNotRetryRegionException(ri.getEncodedName() +
-          " is not online (splitParent=true)");
-    }
-    if (ri.isSplit()) {
-      throw new DoNotRetryRegionException(ri.getEncodedName() + " has split=true");
-    }
-    if (ri.isOffline()) {
-      // RegionOfflineException is not instance of DNRIOE so wrap it.
-      throw new DoNotRetryRegionException(new RegionOfflineException(ri.getEncodedName()));
-    }
+    regionNode.checkOnline();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 faad3dd..bd6c371 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
@@ -160,7 +160,7 @@ public class CreateTableProcedure
 
   @Override
   protected CreateTableState getState(final int stateId) {
-    return CreateTableState.valueOf(stateId);
+    return CreateTableState.forNumber(stateId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 060af01..46dca20 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
@@ -177,7 +177,7 @@ public class DeleteTableProcedure
 
   @Override
   protected DeleteTableState getState(final int stateId) {
-    return DeleteTableState.valueOf(stateId);
+    return DeleteTableState.forNumber(stateId);
   }
 
   @Override
@@ -191,6 +191,11 @@ public class DeleteTableProcedure
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  @Override
   public TableName getTableName() {
     return tableName;
   }
@@ -297,7 +302,9 @@ public class DeleteTableProcedure
           FileStatus[] files = fs.listStatus(tempdir);
           if (files != null && files.length > 0) {
             for (int i = 0; i < files.length; ++i) {
-              if (!files[i].isDir()) continue;
+              if (!files[i].isDirectory()) {
+                continue;
+              }
               HFileArchiver.archiveRegion(fs, mfs.getRootDir(), tempTableDir, files[i].getPath());
             }
           }
@@ -343,7 +350,6 @@ public class DeleteTableProcedure
    * There may be items for this table still up in hbase:meta in the case where the
    * info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta
    * that have to do with this table. See HBASE-12980.
-   * @throws IOException
    */
   private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 3a2a952..71d1fc9 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
@@ -51,8 +51,6 @@ public class DisableTableProcedure
   private TableName tableName;
   private boolean skipTableStateCheck;
 
-  private Boolean traceEnabled = null;
-
   public DisableTableProcedure() {
     super();
   }
@@ -64,8 +62,7 @@ public class DisableTableProcedure
    * @param skipTableStateCheck whether to check table state
    */
   public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
-      final boolean skipTableStateCheck)
-  throws HBaseIOException {
+      final boolean skipTableStateCheck) throws HBaseIOException {
     this(env, tableName, skipTableStateCheck, null);
   }
 
@@ -77,7 +74,7 @@ public class DisableTableProcedure
    */
   public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
       final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch)
-  throws HBaseIOException {
+      throws HBaseIOException {
     super(env, syncLatch);
     this.tableName = tableName;
     preflightChecks(env, true);
@@ -107,7 +104,8 @@ public class DisableTableProcedure
           setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
           break;
         case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
-          addChildProcedure(env.getAssignmentManager().createUnassignProcedures(tableName));
+          addChildProcedure(
+            env.getAssignmentManager().createUnassignProceduresForDisabling(tableName));
           setNextState(DisableTableState.DISABLE_TABLE_ADD_REPLICATION_BARRIER);
           break;
         case DISABLE_TABLE_ADD_REPLICATION_BARRIER:
@@ -180,7 +178,7 @@ public class DisableTableProcedure
 
   @Override
   protected DisableTableState getState(final int stateId) {
-    return DisableTableState.valueOf(stateId);
+    return DisableTableState.forNumber(stateId);
   }
 
   @Override
@@ -219,6 +217,14 @@ public class DisableTableProcedure
     skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
   }
 
+  // For disabling a table, we does not care whether a region can be online so hold the table xlock
+  // for ever. This will simplify the logic as we will not be conflict with procedures other than
+  // SCP.
+  @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
   @Override
   public TableName getTableName() {
     return tableName;
@@ -233,7 +239,6 @@ public class DisableTableProcedure
    * Action before any real action of disabling table. Set the exception in the procedure instead
    * of throwing it.  This approach is to deal with backward compatible with 1.0.
    * @param env MasterProcedureEnv
-   * @throws IOException
    */
   private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
     boolean canTableBeDisabled = true;
@@ -272,8 +277,6 @@ public class DisableTableProcedure
    * Action before disabling table.
    * @param env MasterProcedureEnv
    * @param state the procedure state
-   * @throws IOException
-   * @throws InterruptedException
    */
   protected void preDisable(final MasterProcedureEnv env, final DisableTableState state)
       throws IOException, InterruptedException {
@@ -283,14 +286,11 @@ public class DisableTableProcedure
   /**
    * Mark table state to Disabling
    * @param env MasterProcedureEnv
-   * @throws IOException
    */
-  protected static void setTableStateToDisabling(
-      final MasterProcedureEnv env,
+  private static void setTableStateToDisabling(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
     // Set table disabling flag up in zk.
-    env.getMasterServices().getTableStateManager().setTableState(
-      tableName,
+    env.getMasterServices().getTableStateManager().setTableState(tableName,
       TableState.State.DISABLING);
     LOG.info("Set {} to state={}", tableName, TableState.State.DISABLING);
   }
@@ -298,14 +298,11 @@ public class DisableTableProcedure
   /**
    * Mark table state to Disabled
    * @param env MasterProcedureEnv
-   * @throws IOException
    */
-  protected static void setTableStateToDisabled(
-      final MasterProcedureEnv env,
+  protected static void setTableStateToDisabled(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
     // Flip the table to disabled
-    env.getMasterServices().getTableStateManager().setTableState(
-      tableName,
+    env.getMasterServices().getTableStateManager().setTableState(tableName,
       TableState.State.DISABLED);
     LOG.info("Set {} to state={}", tableName, TableState.State.DISABLED);
   }
@@ -314,8 +311,6 @@ public class DisableTableProcedure
    * Action after disabling table.
    * @param env MasterProcedureEnv
    * @param state the procedure state
-   * @throws IOException
-   * @throws InterruptedException
    */
   protected void postDisable(final MasterProcedureEnv env, final DisableTableState state)
       throws IOException, InterruptedException {
@@ -323,23 +318,9 @@ public class DisableTableProcedure
   }
 
   /**
-   * 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;
-  }
-
-  /**
    * Coprocessor Action.
    * @param env MasterProcedureEnv
    * @param state the procedure state
-   * @throws IOException
-   * @throws InterruptedException
    */
   private void runCoprocessorAction(final MasterProcedureEnv env, final DisableTableState state)
       throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 f2fbb7a..4e6211e 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
@@ -84,7 +84,6 @@ public class EnableTableProcedure
     this.skipTableStateCheck = skipTableStateCheck;
   }
 
-  @SuppressWarnings("deprecation")
   @Override
   protected Flow executeFromState(final MasterProcedureEnv env, final EnableTableState state)
       throws InterruptedException {
@@ -255,7 +254,7 @@ public class EnableTableProcedure
 
   @Override
   protected EnableTableState getState(final int stateId) {
-    return EnableTableState.valueOf(stateId);
+    return EnableTableState.forNumber(stateId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
index d984632..024f3ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
@@ -18,10 +18,11 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.concurrent.CountDownLatch;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
@@ -32,7 +33,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.I
 
 /**
  * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an
- * {@link AssignProcedure} to assign meta.
+ * {@link TransitRegionStateProcedure} to assign meta.
  */
 @InterfaceAudience.Private
 public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMetaState> {
@@ -55,7 +56,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
     switch (state) {
       case INIT_META_ASSIGN_META:
         addChildProcedure(env.getAssignmentManager()
-          .createAssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
+          .createAssignProcedures(Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO)));
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
index 3b848fa..17b5ae3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
@@ -18,37 +18,23 @@
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
-import java.util.Set;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.master.MasterServices;
-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.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-
 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.RecoverMetaState;
 
-
 /**
- * This procedure recovers meta from prior shutdown/ crash of a server, and brings meta online by
- * assigning meta region/s. Any place where meta is accessed and requires meta to be online, need to
- * submit this procedure instead of duplicating steps to recover meta in the code.
- * <p/>
+ * Leave here only for checking if we can successfully start the master.
  * @deprecated Do not use any more, leave it here only for compatible. The recovery work will be
  *             done in {@link ServerCrashProcedure} directly, and the initial work for meta table
  *             will be done by {@link InitMetaProcedure}.
@@ -66,146 +52,17 @@ public class RecoverMetaProcedure
   private boolean shouldSplitWal;
   private int replicaId;
 
-  private final ProcedurePrepareLatch syncLatch;
   private MasterServices master;
 
-  /**
-   * Call this constructor to queue up a {@link RecoverMetaProcedure} in response to meta
-   * carrying server crash
-   * @param failedMetaServer failed/ crashed region server that was carrying meta
-   * @param shouldSplitLog split log file of meta region
-   */
-  public RecoverMetaProcedure(final ServerName failedMetaServer, final boolean shouldSplitLog) {
-    this(failedMetaServer, shouldSplitLog, null);
-  }
-
-  /**
-   * Constructor with latch, for blocking/ sync usage
-   */
-  public RecoverMetaProcedure(final ServerName failedMetaServer, final boolean shouldSplitLog,
-                              final ProcedurePrepareLatch latch) {
-    this.failedMetaServer = failedMetaServer;
-    this.shouldSplitWal = shouldSplitLog;
-    this.replicaId = RegionInfo.DEFAULT_REPLICA_ID;
-    this.syncLatch = latch;
-  }
-
-  /**
-   * This constructor is also used when deserializing from a procedure store; we'll construct one
-   * of these then call #deserializeStateData(InputStream). Do not use directly.
-   */
   public RecoverMetaProcedure() {
-    this(null, false);
+
   }
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env,
       MasterProcedureProtos.RecoverMetaState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
-    prepare(env);
-
-    if (!isRunRequired()) {
-      LOG.info(this + "; Meta already initialized. Skipping run");
-      return Flow.NO_MORE_STATE;
-    }
-
-    try {
-      switch (state) {
-        case RECOVER_META_PREPARE:
-          // If Master is going down or cluster is up, skip this assign by returning NO_MORE_STATE
-          if (!master.isClusterUp()) {
-            String msg = "Cluster not up! Skipping hbase:meta assign.";
-            LOG.warn(msg);
-            return Flow.NO_MORE_STATE;
-          }
-          if (master.isStopping() || master.isStopped()) {
-            String msg = "Master stopping=" + master.isStopping() + ", stopped=" +
-                master.isStopped() + "; skipping hbase:meta assign.";
-            LOG.warn(msg);
-            return Flow.NO_MORE_STATE;
-          }
-          setNextState(RecoverMetaState.RECOVER_META_SPLIT_LOGS);
-          break;
-        case RECOVER_META_SPLIT_LOGS:
-          LOG.info("Start " + this);
-          if (shouldSplitWal) {
-            // TODO: Matteo. We BLOCK here but most important thing to be doing at this moment.
-            AssignmentManager am = env.getMasterServices().getAssignmentManager();
-            if (failedMetaServer != null) {
-              am.getRegionStates().metaLogSplitting(failedMetaServer);
-              master.getMasterWalManager().splitMetaLog(failedMetaServer);
-              am.getRegionStates().metaLogSplit(failedMetaServer);
-            } else {
-              ServerName serverName =
-                  master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper());
-              Set<ServerName> previouslyFailedServers =
-                  master.getMasterWalManager().getFailedServersFromLogFolders();
-              if (serverName != null && previouslyFailedServers.contains(serverName)) {
-                am.getRegionStates().metaLogSplitting(serverName);
-                master.getMasterWalManager().splitMetaLog(serverName);
-                am.getRegionStates().metaLogSplit(serverName);
-              }
-            }
-          }
-          setNextState(RecoverMetaState.RECOVER_META_ASSIGN_REGIONS);
-          break;
-        case RECOVER_META_ASSIGN_REGIONS:
-          RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-              RegionInfoBuilder.FIRST_META_REGIONINFO, this.replicaId);
-
-          AssignProcedure metaAssignProcedure;
-          AssignmentManager am = master.getAssignmentManager();
-          if (failedMetaServer != null) {
-            handleRIT(env, hri, this.failedMetaServer);
-            LOG.info(this + "; Assigning meta with new plan; previous server=" + failedMetaServer);
-            metaAssignProcedure = am.createAssignProcedure(hri);
-          } else {
-            // get server carrying meta from zk
-            ServerName metaServer =
-                MetaTableLocator.getMetaRegionState(master.getZooKeeper()).getServerName();
-            LOG.info(this + "; Retaining meta assignment to server=" + metaServer);
-            metaAssignProcedure = am.createAssignProcedure(hri, metaServer);
-          }
-
-          addChildProcedure(metaAssignProcedure);
-          return Flow.NO_MORE_STATE;
-
-        default:
-          throw new UnsupportedOperationException("unhandled state=" + state);
-      }
-    } catch (IOException|KeeperException e) {
-      LOG.warn(this + "; Failed state=" + state + ", retry " + this + "; cycles=" +
-          getCycles(), e);
-    }
-    return Flow.HAS_MORE_STATE;
-  }
-
-  /**
-   * Is the region stuck assigning to this failedMetaServer? If so, cancel the call
-   * just as we do over in ServerCrashProcedure#handleRIT except less to do here; less context
-   * to carry.
-   */
-  // NOTE: Make sure any fix or improvement done here is also done in SCP#handleRIT; the methods
-  // have overlap.
-  private void handleRIT(MasterProcedureEnv env, RegionInfo ri, ServerName crashedServerName) {
-    AssignmentManager am = env.getAssignmentManager();
-    RegionTransitionProcedure rtp = am.getRegionStates().getRegionTransitionProcedure(ri);
-    if (rtp == null) {
-      return; // Nothing to do. Not in RIT.
-    }
-    // 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);
-    } else if (rtpServerName.equals(crashedServerName)) {
-      LOG.info("pid=" + getProcId() + " found RIT " + rtp + "; " +
-          rtp.getRegionState(env).toShortString());
-      rtp.remoteCallFailed(env, crashedServerName,
-          new ServerCrashException(getProcId(), crashedServerName));
-    }
+    return Flow.NO_MORE_STATE;
   }
 
   @Override
@@ -241,11 +98,10 @@ public class RecoverMetaProcedure
   }
 
   @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     MasterProcedureProtos.RecoverMetaStateData.Builder state =
-        MasterProcedureProtos.RecoverMetaStateData.newBuilder().setShouldSplitWal(shouldSplitWal);
+      MasterProcedureProtos.RecoverMetaStateData.newBuilder().setShouldSplitWal(shouldSplitWal);
     if (failedMetaServer != null) {
       state.setFailedMetaServer(ProtobufUtil.toServerName(failedMetaServer));
     }
@@ -254,50 +110,13 @@ public class RecoverMetaProcedure
   }
 
   @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
     MasterProcedureProtos.RecoverMetaStateData state =
-        serializer.deserialize(MasterProcedureProtos.RecoverMetaStateData.class);
+      serializer.deserialize(MasterProcedureProtos.RecoverMetaStateData.class);
     this.shouldSplitWal = state.hasShouldSplitWal() && state.getShouldSplitWal();
-    this.failedMetaServer = state.hasFailedMetaServer() ?
-        ProtobufUtil.toServerName(state.getFailedMetaServer()) : null;
+    this.failedMetaServer =
+      state.hasFailedMetaServer() ? ProtobufUtil.toServerName(state.getFailedMetaServer()) : null;
     this.replicaId = state.hasReplicaId() ? state.getReplicaId() : RegionInfo.DEFAULT_REPLICA_ID;
   }
-
-  @Override
-  protected LockState acquireLock(MasterProcedureEnv env) {
-    if (env.getProcedureScheduler().waitMetaExclusiveLock(this)) {
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(MasterProcedureEnv env) {
-    env.getProcedureScheduler().wakeMetaExclusiveLock(this);
-  }
-
-  @Override
-  protected void completionCleanup(MasterProcedureEnv env) {
-    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
-  }
-
-  /**
-   * @return true if failedMetaServer is not null (meta carrying server crashed) or meta is
-   * already initialized
-   */
-  private boolean isRunRequired() {
-    return failedMetaServer != null || !master.getAssignmentManager().isMetaAssigned();
-  }
-
-  /**
-   * Prepare for execution
-   */
-  private void prepare(MasterProcedureEnv env) {
-    if (master == null) {
-      master = env.getMasterServices();
-      Preconditions.checkArgument(master != null);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
index 8f3aa22..9ccbc78 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ReopenTableRegionsProcedure.java
@@ -21,11 +21,10 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.master.assignment.MoveRegionProcedure;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
@@ -39,8 +38,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R
 
 /**
  * Used for reopening the regions for a table.
- * <p/>
- * Currently we use {@link MoveRegionProcedure} to reopen regions.
  */
 @InterfaceAudience.Private
 public class ReopenTableRegionsProcedure
@@ -69,16 +66,6 @@ public class ReopenTableRegionsProcedure
     return TableOperationType.REGION_EDIT;
   }
 
-  private MoveRegionProcedure createReopenProcedure(MasterProcedureEnv env, HRegionLocation loc) {
-    try {
-      return new MoveRegionProcedure(env,
-        new RegionPlan(loc.getRegion(), loc.getServerName(), loc.getServerName()), false);
-    } catch (HBaseIOException e) {
-      // we skip the checks so this should not happen
-      throw new AssertionError(e);
-    }
-  }
-
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, ReopenTableRegionsState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
@@ -93,8 +80,22 @@ public class ReopenTableRegionsProcedure
         setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_REOPEN_REGIONS);
         return Flow.HAS_MORE_STATE;
       case REOPEN_TABLE_REGIONS_REOPEN_REGIONS:
-        addChildProcedure(regions.stream().filter(l -> l.getSeqNum() >= 0)
-          .map(l -> createReopenProcedure(env, l)).toArray(MoveRegionProcedure[]::new));
+        for (HRegionLocation loc : regions) {
+          RegionStateNode regionNode = env.getAssignmentManager().getRegionStates()
+            .getOrCreateRegionStateNode(loc.getRegion());
+          TransitRegionStateProcedure proc;
+          regionNode.lock();
+          try {
+            if (regionNode.getProcedure() != null) {
+              continue;
+            }
+            proc = TransitRegionStateProcedure.reopen(env, regionNode.getRegionInfo());
+            regionNode.setProcedure(proc);
+          } finally {
+            regionNode.unlock();
+          }
+          addChildProcedure(proc);
+        }
         setNextState(ReopenTableRegionsState.REOPEN_TABLE_REGIONS_CONFIRM_REOPENED);
         return Flow.HAS_MORE_STATE;
       case REOPEN_TABLE_REGIONS_CONFIRM_REOPENED:


[2/7] hbase git commit: HBASE-20881 Introduce a region transition procedure to handle all the state transition for a region

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
new file mode 100644
index 0000000..1c97f37
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
@@ -0,0 +1,586 @@
+/**
+ * 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.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.SocketTimeoutException;
+import java.util.Arrays;
+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.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+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.ProcedureMetrics;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+
+/**
+ * Base class for AM test.
+ */
+public abstract class TestAssignmentManagerBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentManagerBase.class);
+
+  @Rule
+  public TestName name = new TestName();
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  protected static final int PROC_NTHREADS = 64;
+  protected static final int NREGIONS = 1 * 1000;
+  protected static final int NSERVERS = Math.max(1, NREGIONS / 100);
+
+  protected HBaseTestingUtility util;
+  protected MockRSProcedureDispatcher rsDispatcher;
+  protected MockMasterServices master;
+  protected AssignmentManager am;
+  protected NavigableMap<ServerName, SortedSet<byte[]>> regionsToRegionServers =
+    new ConcurrentSkipListMap<ServerName, SortedSet<byte[]>>();
+  // Simple executor to run some simple tasks.
+  protected ScheduledExecutorService executor;
+
+  protected ProcedureMetrics assignProcMetrics;
+  protected ProcedureMetrics unassignProcMetrics;
+
+  protected long assignSubmittedCount = 0;
+  protected long assignFailedCount = 0;
+  protected long unassignSubmittedCount = 0;
+  protected long unassignFailedCount = 0;
+
+  protected int getAssignMaxAttempts() {
+    // Have many so we succeed eventually.
+    return 100;
+  }
+
+  protected 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, getAssignMaxAttempts());
+  }
+
+  @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();
+    assignProcMetrics = am.getAssignmentManagerMetrics().getAssignProcMetrics();
+    unassignProcMetrics = am.getAssignmentManagerMetrics().getUnassignProcMetrics();
+    setUpMeta();
+  }
+
+  protected void setUpMeta() throws Exception {
+    rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+    am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
+    am.wakeMetaLoadedEvent();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    master.stop("tearDown");
+    this.executor.shutdownNow();
+  }
+
+  protected class NoopRsExecutor implements MockRSExecutor {
+    @Override
+    public ExecuteProceduresResponse sendRequest(ServerName server,
+        ExecuteProceduresRequest request) throws IOException {
+      if (request.getOpenRegionCount() > 0) {
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          for (RegionOpenInfo openReq : req.getOpenInfoList()) {
+            execOpenRegion(server, openReq);
+          }
+        }
+      }
+      if (request.getCloseRegionCount() > 0) {
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          execCloseRegion(server, req.getRegion().getValue().toByteArray());
+        }
+      }
+      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;
+    }
+  }
+
+  protected Future<byte[]> submitProcedure(final Procedure<MasterProcedureEnv> proc) {
+    return ProcedureSyncWait.submitProcedure(master.getMasterProcedureExecutor(), proc);
+  }
+
+  protected byte[] waitOnFuture(final Future<byte[]> future) throws Exception {
+    try {
+      return future.get(5, TimeUnit.SECONDS);
+    } catch (ExecutionException e) {
+      LOG.info("ExecutionException", e);
+      Exception ee = (Exception) e.getCause();
+      if (ee instanceof InterruptedIOException) {
+        for (Procedure<?> p : this.master.getMasterProcedureExecutor().getProcedures()) {
+          LOG.info(p.toStringDetails());
+        }
+      }
+      throw (Exception) e.getCause();
+    }
+  }
+
+  // ============================================================================================
+  // Helpers
+  // ============================================================================================
+  protected void bulkSubmit(TransitRegionStateProcedure[] procs) throws Exception {
+    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);
+    }
+  }
+
+  protected TransitRegionStateProcedure createAndSubmitAssign(TableName tableName, int regionId) {
+    RegionInfo hri = createRegionInfo(tableName, regionId);
+    TransitRegionStateProcedure proc = createAssignProcedure(hri);
+    master.getMasterProcedureExecutor().submitProcedure(proc);
+    return proc;
+  }
+
+  protected RegionInfo createRegionInfo(final TableName tableName, final long regionId) {
+    return RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes(regionId))
+      .setEndKey(Bytes.toBytes(regionId + 1)).setSplit(false).setRegionId(0).build();
+  }
+
+  protected TransitRegionStateProcedure createAssignProcedure(RegionInfo hri) {
+    return am.createAssignProcedures(Arrays.asList(hri))[0];
+  }
+
+  protected TransitRegionStateProcedure createUnassignProcedure(RegionInfo hri) {
+    RegionStateNode regionNode = am.getRegionStates().getRegionStateNode(hri);
+    TransitRegionStateProcedure proc;
+    regionNode.lock();
+    try {
+      assertFalse(regionNode.isInTransition());
+      proc = TransitRegionStateProcedure
+        .unassign(master.getMasterProcedureExecutor().getEnvironment(), hri);
+      regionNode.setProcedure(proc);
+    } finally {
+      regionNode.unlock();
+    }
+    return proc;
+  }
+
+  protected void sendTransitionReport(final ServerName serverName,
+      final org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.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());
+  }
+
+  protected void doCrash(final ServerName serverName) {
+    this.am.submitServerCrash(serverName, false/* No WALs here */);
+  }
+
+  protected void doRestart(final ServerName serverName) {
+    try {
+      this.master.restartRegionServer(serverName);
+    } catch (IOException e) {
+      LOG.warn("Can not restart RS with new startcode");
+    }
+  }
+
+  protected 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);
+      }
+      RegionInfo hri = ProtobufUtil.toRegionInfo(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 {
+      RegionInfo hri = am.getRegionInfo(regionName);
+      sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED);
+      return CloseRegionResponse.newBuilder().setClosed(true).build();
+    }
+  }
+
+  protected static class ServerNotYetRunningRsExecutor implements MockRSExecutor {
+    @Override
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      throw new ServerNotRunningYetException("wait on server startup");
+    }
+  }
+
+  protected static class FaultyRsExecutor implements MockRSExecutor {
+    private final IOException exception;
+
+    public FaultyRsExecutor(final IOException exception) {
+      this.exception = exception;
+    }
+
+    @Override
+    public ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException {
+      throw exception;
+    }
+  }
+
+  protected 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;
+    }
+
+    @Override
+    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.
+   */
+  protected 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;
+    }
+  }
+
+  /**
+   * 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.). Different with
+   * HangThenRSCrashExecutor, HangThenRSCrashExecutor will create ServerCrashProcedure to handle the
+   * server crash. However, this HangThenRSRestartExecutor will restart RS directly, situation for
+   * RS crashed when SCP is not enabled.
+   */
+  protected class HangThenRSRestartExecutor 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("Restarting RS of " + server);
+          doRestart(server);
+        }
+      }, 1, TimeUnit.SECONDS);
+      return null;
+    }
+  }
+
+  protected 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:
+          executor.schedule(new Runnable() {
+            @Override
+            public void run() {
+              LOG.info("Sending in CRASH of " + server);
+              doCrash(server);
+            }
+          }, 1, TimeUnit.SECONDS);
+          throw new RegionServerAbortedException("Fake!");
+        case 2:
+          executor.schedule(new Runnable() {
+            @Override
+            public void run() {
+              LOG.info("Sending in CRASH of " + server);
+              doCrash(server);
+            }
+          }, 1, TimeUnit.SECONDS);
+          throw new RegionServerStoppedException("Fake!");
+        case 3:
+          throw new ServerNotRunningYetException("Fake!");
+        case 4:
+          LOG.info("Returned null from serverName={}; means STUCK...TODO timeout", server);
+          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);
+      }
+    }
+  }
+
+  protected class RandRsExecutor extends NoopRsExecutor {
+    private final Random rand = new Random();
+
+    @Override
+    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");
+        default:
+          // fall out
+      }
+      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;
+        default:
+          // fall out
+      }
+      // 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) {
+        RegionInfo hri = am.getRegionInfo(regionName);
+        sendTransitionReport(server, ProtobufUtil.toRegionInfo(hri), TransitionCode.CLOSED);
+      }
+      resp.setClosed(closed);
+      return resp.build();
+    }
+  }
+
+  protected interface MockRSExecutor {
+    ExecuteProceduresResponse sendRequest(ServerName server, ExecuteProceduresRequest req)
+        throws IOException;
+  }
+
+  protected 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,
+        @SuppressWarnings("rawtypes") Set<RemoteProcedure> remoteProcedures) {
+      submitTask(new MockRemoteCall(serverName, remoteProcedures));
+    }
+
+    private class MockRemoteCall extends ExecuteProceduresRemoteCall {
+      public MockRemoteCall(final ServerName serverName,
+          @SuppressWarnings("rawtypes") final Set<RemoteProcedure> operations) {
+        super(serverName, operations);
+      }
+
+      @Override
+      protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
+          final ExecuteProceduresRequest request) throws IOException {
+        return mockRsExec.sendRequest(serverName, request);
+      }
+    }
+  }
+
+  protected void collectAssignmentManagerMetrics() {
+    assignSubmittedCount = assignProcMetrics.getSubmittedCounter().getCount();
+    assignFailedCount = assignProcMetrics.getFailedCounter().getCount();
+    unassignSubmittedCount = unassignProcMetrics.getSubmittedCounter().getCount();
+    unassignFailedCount = unassignProcMetrics.getFailedCounter().getCount();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java
new file mode 100644
index 0000000..e407faf
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerUtil.java
@@ -0,0 +1,134 @@
+/**
+ * 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.assertFalse;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+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.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestAssignmentManagerUtil {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAssignmentManagerUtil.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("AM");
+
+  private static MasterProcedureEnv ENV;
+
+  private static AssignmentManager AM;
+
+  private static int REGION_REPLICATION = 3;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(1);
+    UTIL.getAdmin().balancerSwitch(false, true);
+    UTIL.createTable(TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf"))
+      .setRegionReplication(REGION_REPLICATION).build(), new byte[][] { Bytes.toBytes(0) });
+    UTIL.waitTableAvailable(TABLE_NAME);
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    ENV = master.getMasterProcedureExecutor().getEnvironment();
+    AM = master.getAssignmentManager();
+  }
+
+  @After
+  public void tearDownAfterTest() throws IOException {
+    for (RegionInfo region : UTIL.getAdmin().getRegions(TABLE_NAME)) {
+      RegionStateNode regionNode = AM.getRegionStates().getRegionStateNode(region);
+      // confirm that we have released the lock
+      assertFalse(((ReentrantLock) regionNode.lock).isLocked());
+      TransitRegionStateProcedure proc = regionNode.getProcedure();
+      if (proc != null) {
+        regionNode.unsetProcedure(proc);
+      }
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  private List<RegionInfo> getPrimaryRegions() throws IOException {
+    return UTIL.getAdmin().getRegions(TABLE_NAME).stream()
+      .filter(r -> RegionReplicaUtil.isDefaultReplica(r)).collect(Collectors.toList());
+  }
+
+  @Test
+  public void testCreateUnassignProcedureForSplitFail() throws IOException {
+    RegionInfo region = getPrimaryRegions().get(0);
+    AM.getRegionStates().getRegionStateNode(region)
+      .setProcedure(TransitRegionStateProcedure.unassign(ENV, region));
+    try {
+      AssignmentManagerUtil.createUnassignProceduresForSplitOrMerge(ENV, Stream.of(region),
+        REGION_REPLICATION);
+      fail("Should fail as the region is in transition");
+    } catch (HBaseIOException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testCreateUnassignProceduresForMergeFail() throws IOException {
+    List<RegionInfo> regions = getPrimaryRegions();
+    RegionInfo regionA = regions.get(0);
+    RegionInfo regionB = regions.get(1);
+    AM.getRegionStates().getRegionStateNode(regionB)
+      .setProcedure(TransitRegionStateProcedure.unassign(ENV, regionB));
+    try {
+      AssignmentManagerUtil.createUnassignProceduresForSplitOrMerge(ENV,
+        Stream.of(regionA, regionB), REGION_REPLICATION);
+      fail("Should fail as the region is in transition");
+    } catch (HBaseIOException e) {
+      // expected
+    }
+    IntStream.range(0, REGION_REPLICATION)
+      .mapToObj(i -> RegionReplicaUtil.getRegionInfoForReplica(regionA, i))
+      .map(AM.getRegionStates()::getRegionStateNode).forEachOrdered(
+        rn -> assertFalse("Should have unset the proc for " + rn, rn.isInTransition()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
new file mode 100644
index 0000000..9b1f4ca
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
@@ -0,0 +1,237 @@
+/**
+ * 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.Iterator;
+import java.util.concurrent.CountDownLatch;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.master.procedure.ServerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+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.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.gson.JsonArray;
+import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+import org.apache.hbase.thirdparty.com.google.gson.JsonParser;
+
+/**
+ * Confirm that we will do backoff when retrying on closing a region, to avoid consuming all the
+ * CPUs.
+ */
+@Category({ MasterTests.class, MediumTests.class })
+public class TestCloseRegionWhileRSCrash {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCloseRegionWhileRSCrash.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestCloseRegionWhileRSCrash.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName TABLE_NAME = TableName.valueOf("Backoff");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static CountDownLatch ARRIVE = new CountDownLatch(1);
+
+  private static CountDownLatch RESUME = new CountDownLatch(1);
+
+  public static final class DummyServerProcedure extends Procedure<MasterProcedureEnv>
+      implements ServerProcedureInterface {
+
+    private ServerName serverName;
+
+    public DummyServerProcedure() {
+    }
+
+    public DummyServerProcedure(ServerName serverName) {
+      this.serverName = serverName;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return serverName;
+    }
+
+    @Override
+    public boolean hasMetaTableRegion() {
+      return false;
+    }
+
+    @Override
+    public ServerOperationType getServerOperationType() {
+      return ServerOperationType.CRASH_HANDLER;
+    }
+
+    @Override
+    protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+        throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+      ARRIVE.countDown();
+      RESUME.await();
+      return null;
+    }
+
+    @Override
+    protected LockState acquireLock(final MasterProcedureEnv env) {
+      if (env.getProcedureScheduler().waitServerExclusiveLock(this, getServerName())) {
+        return LockState.LOCK_EVENT_WAIT;
+      }
+      return LockState.LOCK_ACQUIRED;
+    }
+
+    @Override
+    protected void releaseLock(final MasterProcedureEnv env) {
+      env.getProcedureScheduler().wakeServerExclusiveLock(this, getServerName());
+    }
+
+    @Override
+    protected boolean holdLock(MasterProcedureEnv env) {
+      return true;
+    }
+
+    @Override
+    protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    }
+
+    @Override
+    protected boolean abort(MasterProcedureEnv env) {
+      return false;
+    }
+
+    @Override
+    protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+
+    }
+
+    @Override
+    protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    UTIL.createTable(TABLE_NAME, CF);
+    UTIL.getAdmin().balancerSwitch(false, true);
+    HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
+    if (!srcRs.getRegions(TableName.META_TABLE_NAME).isEmpty()) {
+      RegionInfo metaRegion = srcRs.getRegions(TableName.META_TABLE_NAME).get(0).getRegionInfo();
+      HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs);
+      UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(),
+        Bytes.toBytes(dstRs.getServerName().getServerName()));
+      UTIL.waitFor(30000, () -> !dstRs.getRegions(TableName.META_TABLE_NAME).isEmpty());
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testRetryBackoff() throws IOException, InterruptedException {
+    HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
+    RegionInfo region = srcRs.getRegions(TABLE_NAME).get(0).getRegionInfo();
+    HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs);
+    ProcedureExecutor<MasterProcedureEnv> procExec =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    procExec.submitProcedure(new DummyServerProcedure(srcRs.getServerName()));
+    ARRIVE.await();
+    UTIL.getMiniHBaseCluster().killRegionServer(srcRs.getServerName());
+    UTIL.waitFor(30000,
+      () -> procExec.getProcedures().stream().anyMatch(p -> p instanceof ServerCrashProcedure));
+    Thread t = new Thread(() -> {
+      try {
+        UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
+          Bytes.toBytes(dstRs.getServerName().getServerName()));
+      } catch (IOException e) {
+      }
+    });
+    t.start();
+    JsonParser parser = new JsonParser();
+    long oldTimeout = 0;
+    int timeoutIncrements = 0;
+    // wait until we enter the WAITING_TIMEOUT state
+    UTIL.waitFor(30000, () -> getTimeout(parser, UTIL.getAdmin().getProcedures()) > 0);
+    while (true) {
+      long timeout = getTimeout(parser, UTIL.getAdmin().getProcedures());
+      if (timeout > oldTimeout) {
+        LOG.info("Timeout incremented, was {}, now is {}, increments={}", timeout, oldTimeout,
+          timeoutIncrements);
+        oldTimeout = timeout;
+        timeoutIncrements++;
+        if (timeoutIncrements > 3) {
+          // If we incremented at least twice, break; the backoff is working.
+          break;
+        }
+      }
+      Thread.sleep(1000);
+    }
+    RESUME.countDown();
+    t.join();
+    // Make sure that the region is online, it may not on the original target server, as we will set
+    // forceNewPlan to true if there is a server crash
+    try (Table table = UTIL.getConnection().getTable(TABLE_NAME)) {
+      table.put(new Put(Bytes.toBytes(1)).addColumn(CF, Bytes.toBytes("cq"), Bytes.toBytes(1)));
+    }
+  }
+
+  /**
+   * @param proceduresAsJSON This is String returned by admin.getProcedures call... an array of
+   *          Procedures as JSON.
+   * @return The Procedure timeout value parsed from the TRSP.
+   */
+  private long getTimeout(JsonParser parser, String proceduresAsJSON) {
+    JsonArray array = parser.parse(proceduresAsJSON).getAsJsonArray();
+    Iterator<JsonElement> iterator = array.iterator();
+    while (iterator.hasNext()) {
+      JsonElement element = iterator.next();
+      JsonObject obj = element.getAsJsonObject();
+      String className = obj.get("className").getAsString();
+      String actualClassName = TransitRegionStateProcedure.class.getName();
+      if (className.equals(actualClassName) && obj.has("timeout")) {
+        return obj.get("timeout").getAsLong();
+      }
+    }
+    return -1L;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 094a5a0..36d7583 100644
--- 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
@@ -24,13 +24,14 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-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.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
@@ -61,15 +62,16 @@ public class TestMergeTableRegionsProcedure {
       HBaseClassTestRule.forClass(TestMergeTableRegionsProcedure.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestMergeTableRegionsProcedure.class);
-  @Rule public final TestName name = new TestName();
+  @Rule
+  public final TestName name = new TestName();
 
-  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private 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 final static Configuration conf = UTIL.getConfiguration();
   private static Admin admin;
 
   private AssignmentManager am;
@@ -95,16 +97,12 @@ public class TestMergeTableRegionsProcedure {
   public static void setupCluster() throws Exception {
     setupConf(conf);
     UTIL.startMiniCluster(1);
-    admin = UTIL.getHBaseAdmin();
+    admin = UTIL.getAdmin();
   }
 
   @AfterClass
   public static void cleanupTest() throws Exception {
-    try {
-      UTIL.shutdownMiniCluster();
-    } catch (Exception e) {
-      LOG.warn("failure shutting down cluster", e);
-    }
+    UTIL.shutdownMiniCluster();
   }
 
   @Before
@@ -114,7 +112,7 @@ public class TestMergeTableRegionsProcedure {
         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);
+    admin.balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
     resetProcExecutorTestingKillFlag();
@@ -127,7 +125,7 @@ public class TestMergeTableRegionsProcedure {
   @After
   public void tearDown() throws Exception {
     resetProcExecutorTestingKillFlag();
-    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+    for (TableDescriptor htd: admin.listTableDescriptors()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       UTIL.deleteTable(htd.getTableName());
     }
@@ -233,6 +231,7 @@ public class TestMergeTableRegionsProcedure {
     List<RegionInfo> tableRegions = createTable(tableName);
 
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillIfHasParent(procExec, false);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     RegionInfo[] regionsToMerge = new RegionInfo[2];
@@ -303,10 +302,9 @@ public class TestMergeTableRegionsProcedure {
     assertRegionCount(tableName, initialRegionCount - 1);
   }
 
-  private List<RegionInfo> createTable(final TableName tableName)
-      throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(FAMILY));
+  private List<RegionInfo> createTable(final TableName tableName) throws Exception {
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
     byte[][] splitRows = new byte[initialRegionCount - 1][];
     for (int i = 0; i < splitRows.length; ++i) {
       splitRows[i] = Bytes.toBytes(String.format("%d", i));

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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
index 83c63da..b9fb518 100644
--- 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 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.Waiter;
 import org.apache.hadoop.hbase.client.CompactionState;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureTestingUtility;
@@ -120,7 +120,7 @@ public class TestSplitTableRegionProcedure {
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
 
     // Turn off balancer so it doesn't cut in and mess up our placements.
-    UTIL.getAdmin().setBalancerRunning(false, true);
+    UTIL.getAdmin().balancerSwitch(false, true);
     // Turn off the meta scanner so it don't remove parent on us.
     UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(false);
     am = UTIL.getHBaseCluster().getMaster().getAssignmentManager();
@@ -132,7 +132,7 @@ public class TestSplitTableRegionProcedure {
   @After
   public void tearDown() throws Exception {
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
-    for (HTableDescriptor htd: UTIL.getAdmin().listTables()) {
+    for (TableDescriptor htd : UTIL.getAdmin().listTableDescriptors()) {
       UTIL.deleteTable(htd.getTableName());
     }
   }
@@ -374,7 +374,7 @@ public class TestSplitTableRegionProcedure {
     MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
         true);
     // check that we have only 1 region
-    assertEquals(1, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+    assertEquals(1, UTIL.getAdmin().getRegions(tableName).size());
     List<HRegion> daughters = UTIL.getMiniHBaseCluster().getRegions(tableName);
     assertEquals(1, daughters.size());
     verifyData(daughters.get(0), startRowNum, rowCount,
@@ -398,6 +398,7 @@ public class TestSplitTableRegionProcedure {
     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.setKillIfHasParent(procExec, false);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // collect AM metrics before test

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestTransitRegionStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestTransitRegionStateProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestTransitRegionStateProcedure.java
new file mode 100644
index 0000000..05197f4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestTransitRegionStateProcedure.java
@@ -0,0 +1,164 @@
+/**
+ * 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.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.HMaster;
+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.regionserver.HRegion;
+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.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+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 TestTransitRegionStateProcedure {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestTransitRegionStateProcedure.class);
+
+  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  @Rule
+  public TestName name = new TestName();
+
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.getConfiguration().setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+    UTIL.startMiniCluster(3);
+    UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws IOException, InterruptedException {
+    tableName = TableName.valueOf(name.getMethodName());
+    UTIL.createTable(tableName, CF);
+    UTIL.waitTableAvailable(tableName);
+  }
+
+  private void resetProcExecutorTestingKillFlag() {
+    ProcedureExecutor<MasterProcedureEnv> procExec =
+      UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    resetProcExecutorTestingKillFlag();
+    UTIL.deleteTable(tableName);
+  }
+
+  private void testRecoveryAndDoubleExcution(TransitRegionStateProcedure proc) throws Exception {
+    HMaster master = UTIL.getHBaseCluster().getMaster();
+    AssignmentManager am = master.getAssignmentManager();
+    RegionStateNode regionNode = am.getRegionStates().getRegionStateNode(proc.getRegion());
+    assertFalse(regionNode.isInTransition());
+    regionNode.setProcedure(proc);
+    assertTrue(regionNode.isInTransition());
+    ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+    long procId = procExec.submitProcedure(proc);
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
+    regionNode = am.getRegionStates().getRegionStateNode(proc.getRegion());
+    assertFalse(regionNode.isInTransition());
+  }
+
+  @Test
+  public void testRecoveryAndDoubleExecutionMove() throws Exception {
+    MasterProcedureEnv env =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getEnvironment();
+    HRegion region = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0);
+    long openSeqNum = region.getOpenSeqNum();
+    TransitRegionStateProcedure proc =
+      TransitRegionStateProcedure.move(env, region.getRegionInfo(), null);
+    testRecoveryAndDoubleExcution(proc);
+    HRegion region2 = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0);
+    long openSeqNum2 = region2.getOpenSeqNum();
+    // confirm that the region is successfully opened
+    assertTrue(openSeqNum2 > openSeqNum);
+  }
+
+  @Test
+  public void testRecoveryAndDoubleExecutionReopen() throws Exception {
+    MasterProcedureEnv env =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getEnvironment();
+    HRegionServer rs = UTIL.getRSForFirstRegionInTable(tableName);
+    HRegion region = rs.getRegions(tableName).get(0);
+    long openSeqNum = region.getOpenSeqNum();
+    TransitRegionStateProcedure proc =
+      TransitRegionStateProcedure.reopen(env, region.getRegionInfo());
+    testRecoveryAndDoubleExcution(proc);
+    // should still be on the same RS
+    HRegion region2 = rs.getRegions(tableName).get(0);
+    long openSeqNum2 = region2.getOpenSeqNum();
+    // confirm that the region is successfully opened
+    assertTrue(openSeqNum2 > openSeqNum);
+  }
+
+  @Test
+  public void testRecoveryAndDoubleExecutionUnassignAndAssign() throws Exception {
+    HMaster master = UTIL.getMiniHBaseCluster().getMaster();
+    MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    HRegion region = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0);
+    RegionInfo regionInfo = region.getRegionInfo();
+    long openSeqNum = region.getOpenSeqNum();
+    TransitRegionStateProcedure unassign = TransitRegionStateProcedure.unassign(env, regionInfo);
+    testRecoveryAndDoubleExcution(unassign);
+    AssignmentManager am = master.getAssignmentManager();
+    assertTrue(am.getRegionStates().getRegionState(regionInfo).isClosed());
+
+    TransitRegionStateProcedure assign = TransitRegionStateProcedure.assign(env, regionInfo, null);
+    testRecoveryAndDoubleExcution(assign);
+
+    HRegion region2 = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0);
+    long openSeqNum2 = region2.getOpenSeqNum();
+    // confirm that the region is successfully opened
+    assertTrue(openSeqNum2 > openSeqNum);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestUnexpectedStateException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestUnexpectedStateException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestUnexpectedStateException.java
deleted file mode 100644
index 0f62f8e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestUnexpectedStateException.java
+++ /dev/null
@@ -1,167 +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.fail;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.RegionInfo;
-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.apache.hbase.thirdparty.com.google.gson.JsonArray;
-import org.apache.hbase.thirdparty.com.google.gson.JsonElement;
-import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
-import org.apache.hbase.thirdparty.com.google.gson.JsonParser;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Tests for HBASE-18408 "AM consumes CPU and fills up the logs really fast when there is no RS to
- * assign". If an {@link org.apache.hadoop.hbase.exceptions.UnexpectedStateException}, we'd spin on
- * the ProcedureExecutor consuming CPU and filling logs. Test new back-off facility.
- */
-@Category({MasterTests.class, MediumTests.class})
-public class TestUnexpectedStateException {
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestUnexpectedStateException.class);
-  @Rule public final TestName name = new TestName();
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestUnexpectedStateException.class);
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final byte [] FAMILY = Bytes.toBytes("family");
-  private TableName tableName;
-  private static final int REGIONS = 10;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    TEST_UTIL.startMiniCluster();
-  }
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  @Before
-  public void before() throws IOException {
-    this.tableName = TableName.valueOf(this.name.getMethodName());
-    TEST_UTIL.createMultiRegionTable(this.tableName, FAMILY, REGIONS);
-  }
-
-  private RegionInfo pickArbitraryRegion(Admin admin) throws IOException {
-    List<RegionInfo> regions = admin.getRegions(this.tableName);
-    return regions.get(3);
-  }
-
-  /**
-   * Manufacture a state that will throw UnexpectedStateException.
-   * Change an assigned region's 'state' to be OPENING. That'll mess up a subsequent unassign
-   * causing it to throw UnexpectedStateException. We can easily manufacture this infinite retry
-   * state in UnassignProcedure because it has no startTransition. AssignProcedure does where it
-   * squashes whatever the current region state is making it OFFLINE. That makes it harder to mess
-   * it up. Make do with UnassignProcedure for now.
-   */
-  @Test
-  public void testUnableToAssign() throws Exception {
-    try (Admin admin = TEST_UTIL.getAdmin()) {
-      // Pick a random region from this tests' table to play with. Get its RegionStateNode.
-      // Clone it because the original will be changed by the system. We need clone to fake out
-      // a state.
-      final RegionInfo region = pickArbitraryRegion(admin);
-      AssignmentManager am = TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager();
-      RegionStates.RegionStateNode rsn =  am.getRegionStates().getRegionStateNode(region);
-      // Now force region to be in OPENING state.
-      am.markRegionAsOpening(rsn);
-      // Now the 'region' is in an artificially bad state, try an unassign again.
-      // Run unassign in a thread because it is blocking.
-      Runnable unassign = () -> {
-        try {
-          admin.unassign(region.getRegionName(), true);
-        } catch (IOException ioe) {
-          fail("Failed assign");
-        }
-      };
-      Thread t = new Thread(unassign, "unassign");
-      t.start();
-      while(!t.isAlive()) {
-        Threads.sleep(100);
-      }
-      Threads.sleep(1000);
-      // Unassign should be running and failing. Look for incrementing timeout as evidence that
-      // Unassign is stuck and doing backoff.
-      // Now fix the condition we were waiting on so the unassign can complete.
-      JsonParser parser = new JsonParser();
-      long oldTimeout = 0;
-      int timeoutIncrements = 0;
-      while (true) {
-        long timeout = getUnassignTimeout(parser, admin.getProcedures());
-        if (timeout > oldTimeout) {
-          LOG.info("Timeout incremented, was {}, now is {}, increments={}",
-              timeout, oldTimeout, timeoutIncrements);
-          oldTimeout = timeout;
-          timeoutIncrements++;
-          if (timeoutIncrements > 3) {
-            // If we incremented at least twice, break; the backoff is working.
-            break;
-          }
-        }
-        Thread.sleep(1000);
-      }
-      am.markRegionAsOpened(rsn);
-      t.join();
-    }
-  }
-
-  /**
-   * @param proceduresAsJSON This is String returned by admin.getProcedures call... an array of
-   *                         Procedures as JSON.
-   * @return The Procedure timeout value parsed from the Unassign Procedure.
-   * @Exception Thrown if we do not find UnassignProcedure or fail to parse timeout.
-   */
-  private long getUnassignTimeout(JsonParser parser, String proceduresAsJSON) throws Exception {
-    JsonArray array = parser.parse(proceduresAsJSON).getAsJsonArray();
-    Iterator<JsonElement> iterator = array.iterator();
-    while (iterator.hasNext()) {
-      JsonElement element = iterator.next();
-      JsonObject obj = element.getAsJsonObject();
-      String className = obj.get("className").getAsString();
-      String actualClassName = UnassignProcedure.class.getName();
-      if (className.equals(actualClassName)) {
-        return obj.get("timeout").getAsLong();
-      }
-    }
-    throw new Exception("Failed to find UnassignProcedure or timeout in " + proceduresAsJSON);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 45f3948..a74a3e5 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
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.favored.FavoredNodesPlan;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 26e0956..3975ec0 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
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 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.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -72,13 +74,13 @@ public class MasterProcedureTestingUtility {
   public static void restartMasterProcedureExecutor(ProcedureExecutor<MasterProcedureEnv> procExec)
       throws Exception {
     final MasterProcedureEnv env = procExec.getEnvironment();
-    final HMaster master = (HMaster)env.getMasterServices();
+    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();
+          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());
@@ -88,12 +90,26 @@ public class MasterProcedureTestingUtility {
           return null;
         }
       },
-      // restart services
+      // setup RIT before starting workers
       new Callable<Void>() {
+
         @Override
         public Void call() throws Exception {
-          final AssignmentManager am = env.getAssignmentManager();
+          AssignmentManager am = env.getAssignmentManager();
           am.start();
+          // just follow the same way with HMaster.finishActiveMasterInitialization. See the
+          // comments there
+          am.setupRIT(procExec.getActiveProceduresNoCopy().stream().filter(p -> !p.isSuccess())
+            .filter(p -> p instanceof TransitRegionStateProcedure)
+            .map(p -> (TransitRegionStateProcedure) p).collect(Collectors.toList()));
+          return null;
+        }
+      },
+      // restart services
+      new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          AssignmentManager am = env.getAssignmentManager();
           am.joinCluster();
           master.setInitialized(true);
           return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 bda2c8a..14c8d96 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
@@ -20,13 +20,15 @@ package org.apache.hadoop.hbase.master.procedure;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
+import java.util.stream.Stream;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 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.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
@@ -91,20 +93,18 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
     return 1;
   }
 
-  public static HTableDescriptor createHTableDescriptor(
-      final TableName tableName, final byte[] ... family) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (int i = 0; i < family.length; ++i) {
-      htd.addFamily(new HColumnDescriptor(family[i]));
-    }
-    return htd;
+  public static TableDescriptor createTableDescriptor(TableName tableName, byte[]... family) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    Stream.of(family).map(ColumnFamilyDescriptorBuilder::of)
+      .forEachOrdered(builder::setColumnFamily);
+    return builder.build();
   }
 
   @Test
   public void testCloneSnapshot() throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName clonedTableName = TableName.valueOf("testCloneSnapshot2");
-    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+    final TableDescriptor htd = createTableDescriptor(clonedTableName, CF);
 
     // take the snapshot
     SnapshotProtos.SnapshotDescription snapshotDesc = getSnapshot();
@@ -124,7 +124,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
 
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName clonedTableName = TableName.valueOf(snapshotDesc.getTable());
-    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+    final TableDescriptor htd = createTableDescriptor(clonedTableName, CF);
 
     long procId = ProcedureTestingUtility.submitAndWait(
       procExec, new CloneSnapshotProcedure(procExec.getEnvironment(), htd, snapshotDesc));
@@ -139,11 +139,14 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
   public void testRecoveryAndDoubleExecution() throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName clonedTableName = TableName.valueOf("testRecoveryAndDoubleExecution");
-    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+    final TableDescriptor htd = createTableDescriptor(clonedTableName, CF);
 
     // take the snapshot
     SnapshotProtos.SnapshotDescription snapshotDesc = getSnapshot();
 
+    // Here if you enable this then we will enter an infinite loop, as we will fail either after
+    // TRSP.openRegion or after OpenRegionProcedure.execute, so we can never finish the TRSP...
+    ProcedureTestingUtility.setKillIfHasParent(procExec, false);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // Start the Clone snapshot procedure && kill the executor
@@ -162,7 +165,7 @@ public class TestCloneSnapshotProcedure extends TestTableDDLProcedureBase {
   public void testRollbackAndDoubleExecution() throws Exception {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     final TableName clonedTableName = TableName.valueOf("testRollbackAndDoubleExecution");
-    final HTableDescriptor htd = createHTableDescriptor(clonedTableName, CF);
+    final TableDescriptor htd = createTableDescriptor(clonedTableName, CF);
 
     // take the snapshot
     SnapshotProtos.SnapshotDescription snapshotDesc = getSnapshot();

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 4211187..1eafb29 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
@@ -91,7 +91,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
 
     // Enable the table - expect failure from ProcedurePrepareLatch
     final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
-    long procId3 = procExec.submitProcedure(
+    procExec.submitProcedure(
         new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch));
     prepareLatch.await();
     Assert.fail("Enable should throw exception through latch.");
@@ -108,6 +108,7 @@ public class TestEnableTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
     UTIL.getAdmin().disableTable(tableName);
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillIfHasParent(procExec, false);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // Start the Enable procedure && kill the executor

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRecoverMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRecoverMetaProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRecoverMetaProcedure.java
deleted file mode 100644
index dc939f5..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRecoverMetaProcedure.java
+++ /dev/null
@@ -1,109 +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.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.master.MasterServices;
-import org.apache.hadoop.hbase.master.assignment.MockMasterServices;
-import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
-import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-@Category({MasterTests.class, SmallTests.class})
-public class TestRecoverMetaProcedure {
-  private static final Logger LOG = LoggerFactory.getLogger(TestRecoverMetaProcedure.class);
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestRecoverMetaProcedure.class);
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  /**
-   * Test the new prepare step.
-   * Here we test that our Mock is faking out the precedure well-enough for it to progress past the
-   * first prepare stage.
-   */
-  @Test
-  public void testPrepare() throws ProcedureSuspendedException, ProcedureYieldException,
-      InterruptedException, IOException {
-    RecoverMetaProcedure rmp = new RecoverMetaProcedure();
-    MasterProcedureEnv env = Mockito.mock(MasterProcedureEnv.class);
-    MasterServices masterServices =
-        new MockMasterServices(UTIL.getConfiguration(), null);
-    Mockito.when(env.getMasterServices()).thenReturn(masterServices);
-    assertEquals(StateMachineProcedure.Flow.HAS_MORE_STATE,
-        rmp.executeFromState(env, rmp.getInitialState()));
-    int stateId = rmp.getCurrentStateId();
-    assertEquals(MasterProcedureProtos.RecoverMetaState.RECOVER_META_SPLIT_LOGS_VALUE,
-        rmp.getCurrentStateId());
-  }
-
-  /**
-   * Test the new prepare step.
-   * If Master is stopping, procedure should skip the assign by returning NO_MORE_STATE
-   */
-  @Test
-  public void testPrepareWithMasterStopping() throws ProcedureSuspendedException,
-      ProcedureYieldException, InterruptedException, IOException {
-    RecoverMetaProcedure rmp = new RecoverMetaProcedure();
-    MasterProcedureEnv env = Mockito.mock(MasterProcedureEnv.class);
-    MasterServices masterServices = new MockMasterServices(UTIL.getConfiguration(), null) {
-      @Override
-      public boolean isStopping() {
-        return true;
-      }
-    };
-    Mockito.when(env.getMasterServices()).thenReturn(masterServices);
-    assertEquals(StateMachineProcedure.Flow.NO_MORE_STATE,
-        rmp.executeFromState(env, rmp.getInitialState()));
-  }
-
-  /**
-   * Test the new prepare step.
-   * If cluster is down, procedure should skip the assign by returning NO_MORE_STATE
-   */
-  @Test
-  public void testPrepareWithNoCluster() throws ProcedureSuspendedException,
-      ProcedureYieldException, InterruptedException, IOException {
-    RecoverMetaProcedure rmp = new RecoverMetaProcedure();
-    MasterProcedureEnv env = Mockito.mock(MasterProcedureEnv.class);
-    MasterServices masterServices = new MockMasterServices(UTIL.getConfiguration(), null) {
-      @Override
-      public boolean isClusterUp() {
-        return false;
-      }
-    };
-    Mockito.when(env.getMasterServices()).thenReturn(masterServices);
-    assertEquals(StateMachineProcedure.Flow.NO_MORE_STATE,
-        rmp.executeFromState(env, rmp.getInitialState()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 9f7fafe..f7cc38a 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
@@ -139,6 +139,10 @@ public class TestServerCrashProcedure {
       // Enable test flags and then queue the crash procedure.
       ProcedureTestingUtility.waitNoProcedureRunning(procExec);
       if (doubleExecution) {
+        // For SCP, if you enable this then we will enter an infinite loop, as we will crash between
+        // queue and open for TRSP, and then going back to queue, as we will use the crash rs as the
+        // target server since it is recored in hbase:meta.
+        ProcedureTestingUtility.setKillIfHasParent(procExec, false);
         ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
         // kill the RS
         AssignmentTestingUtil.killRs(util, rsToKill);

http://git-wip-us.apache.org/repos/asf/hbase/blob/bb349413/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 a8f7db5..ea6ee0e 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
@@ -154,7 +154,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
     UTIL.waitUntilAllRegionsAssigned(tableName);
 
     // validate the table regions and layout
-    regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
+    regions = UTIL.getAdmin().getRegions(tableName).toArray(new RegionInfo[0]);
     if (preserveSplits) {
       assertEquals(1 + splitKeys.length, regions.length);
     } else {
@@ -203,6 +203,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
 
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillIfHasParent(procExec, false);
     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
 
     // Start the Truncate procedure && kill the executor
@@ -216,7 +217,7 @@ public class TestTruncateTableProcedure extends TestTableDDLProcedureBase {
     UTIL.waitUntilAllRegionsAssigned(tableName);
 
     // validate the table regions and layout
-    regions = UTIL.getAdmin().getTableRegions(tableName).toArray(new RegionInfo[0]);
+    regions = UTIL.getAdmin().getRegions(tableName).toArray(new RegionInfo[0]);
     if (preserveSplits) {
       assertEquals(1 + splitKeys.length, regions.length);
     } else {