You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/05/12 00:27:28 UTC

[1/4] hbase git commit: Fix CatalogTracker. Make it use Procedures doing clean up of Region data on split/merge. Without these changes, ITBLL was failing at larger scale (3-4hours 5B rows) because we were splitting split Regions.

Repository: hbase
Updated Branches:
  refs/heads/HBASE-14614 9464f461b -> a23fcc97d


http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 9f23848..eca963d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.security.User;
 
 /**
  * Base class for all the Table procedures that want to use a StateMachineProcedure.
- * It provide some basic helpers like basic locking, sync latch, and basic toStringClassDetails().
+ * It provides helpers like basic locking, sync latch, and toStringClassDetails().
  */
 @InterfaceAudience.Private
 public abstract class AbstractStateMachineTableProcedure<TState>
@@ -52,9 +52,10 @@ public abstract class AbstractStateMachineTableProcedure<TState>
 
   protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env,
       final ProcedurePrepareLatch latch) {
-    this.user = env.getRequestUser();
-    this.setOwner(user);
-
+    if (env != null) {
+      this.user = env.getRequestUser();
+      this.setOwner(user);
+    }
     // used for compatibility with clients without procedures
     // they need a sync TableExistsException, TableNotFoundException, TableNotDisabledException, ...
     this.syncLatch = latch;
@@ -110,4 +111,4 @@ public abstract class AbstractStateMachineTableProcedure<TState>
       throw new TableNotFoundException(getTableName());
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
index 1478fc7..15ed429 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DispatchMergingRegionsProcedure.java
@@ -275,7 +275,7 @@ public class DispatchMergingRegionsProcedure
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.MERGE;
+    return TableOperationType.REGION_MERGE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index bcb0004..61e984c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -572,11 +572,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         return false;
       // region operations are using the shared-lock on the table
       // and then they will grab an xlock on the region.
-      case SPLIT:
-      case MERGE:
-      case ASSIGN:
-      case UNASSIGN:
+      case REGION_SPLIT:
+      case REGION_MERGE:
+      case REGION_ASSIGN:
+      case REGION_UNASSIGN:
       case REGION_EDIT:
+      case REGION_GC:
+      case MERGED_REGIONS_GC:
         return false;
       default:
         break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 9e00579..3bd2c9e 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
@@ -413,14 +413,8 @@ implements ServerProcedureInterface {
       final HRegionInfo hri = it.next();
       RegionTransitionProcedure rtp = am.getRegionStates().getRegionTransitionProcedure(hri);
       if (rtp == null) continue;
-      ServerName rtpServerName = rtp.getServer();
-      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());
+      rtp.getRegionState(env).toShortString());
       // Notify RIT on server crash.
       if (sce == null) {
         sce = new ServerCrashException(getProcId());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
index f74df79..86143ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 public interface TableProcedureInterface {
   public enum TableOperationType {
     CREATE, DELETE, DISABLE, EDIT, ENABLE, READ,
-    REGION_EDIT, SPLIT, MERGE, ASSIGN, UNASSIGN, /* region operations */
+    REGION_EDIT, REGION_SPLIT, REGION_MERGE, REGION_ASSIGN, REGION_UNASSIGN,
+      REGION_GC, MERGED_REGIONS_GC/* region operations */
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 43d3ce5..285b529 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -1380,14 +1380,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return !isClosed() && !isClosing();
   }
 
-  /** @return true if region is splittable */
+  @Override
   public boolean isSplittable() {
     return isAvailable() && !hasReferences();
   }
 
-  /**
-   * @return true if region is mergeable
-   */
+  @Override
   public boolean isMergeable() {
     if (!isAvailable()) {
       LOG.debug("Region " + this

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 91b463d..59a0fe5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -318,7 +318,8 @@ public class HRegionFileSystem {
    * @throws IOException
    */
   public boolean hasReferences(final String familyName) throws IOException {
-    FileStatus[] files = FSUtils.listStatus(fs, getStoreDir(familyName));
+    Path storeDir = getStoreDir(familyName);
+    FileStatus[] files = FSUtils.listStatus(fs, storeDir);
     if (files != null) {
       for(FileStatus stat: files) {
         if(stat.isDirectory()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 0c0e500..edad12a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1544,6 +1544,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       if (request.hasCompactionState() && request.getCompactionState()) {
         builder.setCompactionState(region.getCompactionState());
       }
+      builder.setSplittable(region.isSplittable());
+      builder.setMergeable(region.isMergeable());
       builder.setIsRecovering(region.isRecovering());
       return builder.build();
     } catch (IOException ie) {

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 8a46052..b527195 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1744,7 +1744,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public void deleteTable(TableName tableName) throws IOException {
     try {
-      getAdmin().disableTableAsync(tableName);
+      getAdmin().disableTable(tableName);
     } catch (TableNotEnabledException e) {
       LOG.debug("Table: " + tableName + " already disabled, so just deleting it.");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index a4fab7a..074a421 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
@@ -39,11 +38,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
-import org.apache.hadoop.hbase.master.NoSuchProcedureException;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 b3ed2d4..d558aaf 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
@@ -19,16 +19,23 @@ package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.NavigableMap;
 import java.util.SortedSet;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
@@ -41,11 +48,32 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
 import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
+/**
+ * A mocked master services.
+ * Tries to fake it. May not always work.
+ */
 public class MockMasterServices extends MockNoopMasterServices {
   private final MasterFileSystem fileSystemManager;
   private final MasterWalManager walManager;
@@ -54,12 +82,17 @@ public class MockMasterServices extends MockNoopMasterServices {
   private MasterProcedureEnv procedureEnv;
   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
   private ProcedureStore procedureStore;
-
-  private LoadBalancer balancer;
-  private ServerManager serverManager;
+  private final ClusterConnection connection;
+  private final LoadBalancer balancer;
+  private final ServerManager serverManager;
   // Set of regions on a 'server'. Populated externally. Used in below faking 'cluster'.
   private final NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers;
 
+  private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
+  public static final String DEFAULT_COLUMN_FAMILY_NAME = "cf";
+  public static final ServerName MOCK_MASTER_SERVERNAME =
+      ServerName.valueOf("mockmaster.example.org", 1234, -1L);
+
   public MockMasterServices(Configuration conf,
       NavigableMap<ServerName, SortedSet<byte []>> regionsToRegionServers)
   throws IOException {
@@ -68,7 +101,7 @@ public class MockMasterServices extends MockNoopMasterServices {
     Superusers.initialize(conf);
     this.fileSystemManager = new MasterFileSystem(this);
     this.walManager = new MasterWalManager(this);
-    
+    // Mock an AM.
     this.assignmentManager = new AssignmentManager(this, new MockRegionStateStore(this)) {
       public boolean isTableEnabled(final TableName tableName) {
         return true;
@@ -89,16 +122,51 @@ public class MockMasterServices extends MockNoopMasterServices {
     };
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.serverManager = new ServerManager(this);
+
+    // Mock up a Client Interface
+    ClientProtos.ClientService.BlockingInterface ri =
+        Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
+    MutateResponse.Builder builder = MutateResponse.newBuilder();
+    builder.setProcessed(true);
+    try {
+      Mockito.when(ri.mutate((RpcController)Mockito.any(), (MutateRequest)Mockito.any())).
+        thenReturn(builder.build());
+    } catch (ServiceException se) {
+      throw ProtobufUtil.handleRemoteException(se);
+    }
+    try {
+      Mockito.when(ri.multi((RpcController)Mockito.any(), (MultiRequest)Mockito.any())).
+        thenAnswer(new Answer<MultiResponse>() {
+          @Override
+          public MultiResponse answer(InvocationOnMock invocation) throws Throwable {
+            return buildMultiResponse( (MultiRequest)invocation.getArguments()[1]);
+          }
+        });
+    } catch (ServiceException se) {
+      throw ProtobufUtil.getRemoteException(se);
+    }
+    // Mock n ClusterConnection and an AdminProtocol implementation. Have the
+    // ClusterConnection return the HRI.  Have the HRI return a few mocked up responses
+    // to make our test work.
+    this.connection =
+        HConnectionTestingUtility.getMockedConnectionAndDecorate(getConfiguration(),
+          Mockito.mock(AdminProtos.AdminService.BlockingInterface.class), ri, MOCK_MASTER_SERVERNAME,
+          HRegionInfo.FIRST_META_REGIONINFO);
+    // Set hbase.rootdir into test dir.
+    Path rootdir = FSUtils.getRootDir(getConfiguration());
+    FSUtils.setRootDir(getConfiguration(), rootdir);
+    Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
   }
 
   public void start(final int numServes, final RSProcedureDispatcher remoteDispatcher)
       throws IOException {
     startProcedureExecutor(remoteDispatcher);
-    assignmentManager.start();
+    this.assignmentManager.start();
     for (int i = 0; i < numServes; ++i) {
       serverManager.regionServerReport(
         ServerName.valueOf("localhost", 100 + i, 1), ServerLoad.EMPTY_SERVERLOAD);
     }
+    this.procedureExecutor.getEnvironment().setEventReady(initialized, true);
   }
 
   @Override
@@ -115,13 +183,13 @@ public class MockMasterServices extends MockNoopMasterServices {
 
     //procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
     //    new MasterProcedureEnv.WALStoreLeaseRecovery(this));
-    procedureStore = new NoopProcedureStore();
-    procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
+    this.procedureStore = new NoopProcedureStore();
+    this.procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
 
-    procedureEnv = new MasterProcedureEnv(this,
+    this.procedureEnv = new MasterProcedureEnv(this,
        remoteDispatcher != null ? remoteDispatcher : new RSProcedureDispatcher(this));
 
-    procedureExecutor = new ProcedureExecutor(conf, procedureEnv, procedureStore,
+    this.procedureExecutor = new ProcedureExecutor(conf, procedureEnv, procedureStore,
         procedureEnv.getProcedureScheduler());
 
     final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
@@ -130,22 +198,22 @@ public class MockMasterServices extends MockNoopMasterServices {
     final boolean abortOnCorruption = conf.getBoolean(
         MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
         MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
-    procedureStore.start(numThreads);
-    procedureExecutor.start(numThreads, abortOnCorruption);
-    procedureEnv.getRemoteDispatcher().start();
+    this.procedureStore.start(numThreads);
+    this.procedureExecutor.start(numThreads, abortOnCorruption);
+    this.procedureEnv.getRemoteDispatcher().start();
   }
 
   private void stopProcedureExecutor() {
-    if (procedureEnv != null) {
-      procedureEnv.getRemoteDispatcher().stop();
+    if (this.procedureEnv != null) {
+      this.procedureEnv.getRemoteDispatcher().stop();
     }
 
-    if (procedureExecutor != null) {
-      procedureExecutor.stop();
+    if (this.procedureExecutor != null) {
+      this.procedureExecutor.stop();
     }
 
-    if (procedureStore != null) {
-      procedureStore.stop(isAborted());
+    if (this.procedureStore != null) {
+      this.procedureStore.stop(isAborted());
     }
   }
 
@@ -155,6 +223,11 @@ public class MockMasterServices extends MockNoopMasterServices {
   }
 
   @Override
+  public ProcedureEvent getInitializedEvent() {
+    return this.initialized;
+  }
+
+  @Override
   public MasterFileSystem getMasterFileSystem() {
     return fileSystemManager;
   }
@@ -185,6 +258,16 @@ public class MockMasterServices extends MockNoopMasterServices {
   }
 
   @Override
+  public ClusterConnection getConnection() {
+    return this.connection;
+  }
+
+  @Override
+  public ServerName getServerName() {
+    return MOCK_MASTER_SERVERNAME;
+  }
+
+  @Override
   public CoordinatedStateManager getCoordinatedStateManager() {
     return super.getCoordinatedStateManager();
   }
@@ -207,4 +290,69 @@ public class MockMasterServices extends MockNoopMasterServices {
         ServerName lastHost, long openSeqNum, long pid) throws IOException {
     }
   }
+
+  @Override
+  public TableDescriptors getTableDescriptors() {
+    return new TableDescriptors() {
+      @Override
+      public HTableDescriptor remove(TableName tablename) throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override
+      public Map<String, HTableDescriptor> getAll() throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override public Map<String, HTableDescriptor> getAllDescriptors() throws IOException {
+        // noop
+        return null;
+      }
+
+      @Override
+      public HTableDescriptor get(TableName tablename) throws IOException {
+        HTableDescriptor htd = new HTableDescriptor(tablename);
+        htd.addFamily(new HColumnDescriptor(DEFAULT_COLUMN_FAMILY_NAME));
+        return htd;
+      }
+
+      @Override
+      public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
+        return null;
+      }
+
+      @Override
+      public void add(HTableDescriptor htd) throws IOException {
+        // noop
+      }
+
+      @Override
+      public void setCacheOn() throws IOException {
+      }
+
+      @Override
+      public void setCacheOff() throws IOException {
+      }
+    };
+  }
+
+  private static MultiResponse buildMultiResponse(MultiRequest req) {
+    MultiResponse.Builder builder = MultiResponse.newBuilder();
+    RegionActionResult.Builder regionActionResultBuilder =
+        RegionActionResult.newBuilder();
+    ResultOrException.Builder roeBuilder = ResultOrException.newBuilder();
+    for (RegionAction regionAction: req.getRegionActionList()) {
+      regionActionResultBuilder.clear();
+      for (ClientProtos.Action action: regionAction.getActionList()) {
+        roeBuilder.clear();
+        roeBuilder.setResult(ClientProtos.Result.getDefaultInstance());
+        roeBuilder.setIndex(action.getIndex());
+        regionActionResultBuilder.addResultOrException(roeBuilder.build());
+      }
+      builder.addRegionActionResult(regionActionResultBuilder.build());
+    }
+    return builder.build();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 8be1be9..44fd575 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
@@ -26,11 +26,13 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
@@ -41,18 +43,23 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 @Category({MasterTests.class, MediumTests.class})
-@Ignore // Fix for AMv2.
 public class TestMergeTableRegionsProcedure {
   private static final Log LOG = LogFactory.getLog(TestMergeTableRegionsProcedure.class);
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
+  @Rule public final TestName name = new TestName();
 
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static long nonceGroup = HConstants.NO_NONCE;
@@ -68,7 +75,6 @@ public class TestMergeTableRegionsProcedure {
     conf.setInt("hbase.assignment.maximum.attempts", 3);
     conf.setInt("hbase.master.maximum.ping.server.attempts", 3);
     conf.setInt("hbase.master.ping.server.retry.sleep.interval", 1);
-
     conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
   }
 
@@ -119,9 +125,9 @@ public class TestMergeTableRegionsProcedure {
   /**
    * This tests two region merges
    */
-  @Test(timeout=60000)
+  @Test
   public void testMergeTwoRegions() throws Exception {
-    final TableName tableName = TableName.valueOf("testMergeTwoRegions");
+    final TableName tableName = TableName.valueOf(this.name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     List<HRegionInfo> tableRegions = createTable(tableName);
@@ -129,19 +135,33 @@ public class TestMergeTableRegionsProcedure {
     HRegionInfo[] regionsToMerge = new HRegionInfo[2];
     regionsToMerge[0] = tableRegions.get(0);
     regionsToMerge[1] = tableRegions.get(1);
-
-    long procId = procExec.submitProcedure(new MergeTableRegionsProcedure(
-      procExec.getEnvironment(), regionsToMerge, true));
+    MergeTableRegionsProcedure proc =
+        new MergeTableRegionsProcedure(procExec.getEnvironment(), regionsToMerge, true);
+    long procId = procExec.submitProcedure(proc);
     ProcedureTestingUtility.waitProcedure(procExec, procId);
     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
-
     assertRegionCount(tableName, initialRegionCount - 1);
+    Pair<HRegionInfo, HRegionInfo> pair =
+      MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
+        proc.getMergedRegion().getRegionName());
+    assertTrue(pair.getFirst() != null && pair.getSecond() != null);
+
+    // Can I purge the merged regions from hbase:meta? Check that all went
+    // well by looking at the merged row up in hbase:meta. It should have no
+    // more mention of the merged regions; they are purged as last step in
+    // the merged regions cleanup.
+    UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);
+    UTIL.getHBaseCluster().getMaster().getCatalogJanitor().triggerNow();
+    while (pair != null && pair.getFirst() != null && pair.getSecond() != null) {
+      pair = MetaTableAccessor.getRegionsFromMergeQualifier(UTIL.getConnection(),
+          proc.getMergedRegion().getRegionName());
+    }
   }
 
   /**
    * This tests two concurrent region merges
    */
-  @Test(timeout=60000)
+  @Test
   public void testMergeRegionsConcurrently() throws Exception {
     final TableName tableName = TableName.valueOf("testMergeRegionsConcurrently");
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -166,7 +186,7 @@ public class TestMergeTableRegionsProcedure {
     assertRegionCount(tableName, initialRegionCount - 2);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testRecoveryAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -190,7 +210,7 @@ public class TestMergeTableRegionsProcedure {
     assertRegionCount(tableName, initialRegionCount - 1);
   }
 
-  @Test(timeout = 60000)
+  @Test
   public void testRollbackAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 78836e8..6824597 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
@@ -67,8 +67,8 @@ import org.junit.rules.TestRule;
 @Category({MasterTests.class, MediumTests.class})
 public class TestSplitTableRegionProcedure {
   private static final Log LOG = LogFactory.getLog(TestSplitTableRegionProcedure.class);
-  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
-      withLookingForStuckThread(true).build();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
@@ -143,7 +143,7 @@ public class TestSplitTableRegionProcedure {
     verify(tableName, splitRowNum);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionNoStoreFile() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -167,7 +167,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(tableName) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionUnevenDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -192,7 +192,7 @@ public class TestSplitTableRegionProcedure {
     verify(tableName, splitRowNum);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionEmptyDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -221,7 +221,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testSplitTableRegionDeletedRowsDaughter() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -261,7 +261,7 @@ public class TestSplitTableRegionProcedure {
     assertTrue(UTIL.countRows(daughters.get(0)) == 0 || UTIL.countRows(daughters.get(1)) == 0);
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testInvalidSplitKey() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -284,7 +284,7 @@ public class TestSplitTableRegionProcedure {
     }
   }
 
-  @Test(timeout = 60000)
+  @Test
   public void testRollbackAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -318,7 +318,7 @@ public class TestSplitTableRegionProcedure {
     Bytes.toBytes(ColumnFamilyName1), Bytes.toBytes(ColumnFamilyName2));
   }
 
-  @Test(timeout=60000)
+  @Test
   public void testRecoveryAndDoubleExecution() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
@@ -425,4 +425,4 @@ public class TestSplitTableRegionProcedure {
   private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
     return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
   }
-}
+}
\ No newline at end of file

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

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


[2/4] hbase git commit: Fix CatalogTracker. Make it use Procedures doing clean up of Region data on split/merge. Without these changes, ITBLL was failing at larger scale (3-4hours 5B rows) because we were splitting split Regions.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
index 64732a7..4e3e784 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProtos.java
@@ -39686,10 +39686,18 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
     /**
+     * <pre>
+     * This is how many archiving tasks we started as a result of this scan.
+     * </pre>
+     *
      * <code>optional int32 scan_result = 1;</code>
      */
     boolean hasScanResult();
     /**
+     * <pre>
+     * This is how many archiving tasks we started as a result of this scan.
+     * </pre>
+     *
      * <code>optional int32 scan_result = 1;</code>
      */
     int getScanResult();
@@ -39770,12 +39778,20 @@ public final class MasterProtos {
     public static final int SCAN_RESULT_FIELD_NUMBER = 1;
     private int scanResult_;
     /**
+     * <pre>
+     * This is how many archiving tasks we started as a result of this scan.
+     * </pre>
+     *
      * <code>optional int32 scan_result = 1;</code>
      */
     public boolean hasScanResult() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
+     * <pre>
+     * This is how many archiving tasks we started as a result of this scan.
+     * </pre>
+     *
      * <code>optional int32 scan_result = 1;</code>
      */
     public int getScanResult() {
@@ -40069,18 +40085,30 @@ public final class MasterProtos {
 
       private int scanResult_ ;
       /**
+       * <pre>
+       * This is how many archiving tasks we started as a result of this scan.
+       * </pre>
+       *
        * <code>optional int32 scan_result = 1;</code>
        */
       public boolean hasScanResult() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
+       * <pre>
+       * This is how many archiving tasks we started as a result of this scan.
+       * </pre>
+       *
        * <code>optional int32 scan_result = 1;</code>
        */
       public int getScanResult() {
         return scanResult_;
       }
       /**
+       * <pre>
+       * This is how many archiving tasks we started as a result of this scan.
+       * </pre>
+       *
        * <code>optional int32 scan_result = 1;</code>
        */
       public Builder setScanResult(int value) {
@@ -40090,6 +40118,10 @@ public final class MasterProtos {
         return this;
       }
       /**
+       * <pre>
+       * This is how many archiving tasks we started as a result of this scan.
+       * </pre>
+       *
        * <code>optional int32 scan_result = 1;</code>
        */
       public Builder clearScanResult() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 6e851e6..2e64684 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -38,6 +38,10 @@ message GetRegionInfoResponse {
   required RegionInfo region_info = 1;
   optional CompactionState compaction_state = 2;
   optional bool isRecovering = 3;
+  // True if region is splittable, false otherwise.
+  optional bool splittable = 4;
+  // True if region is mergeable, false otherwise.
+  optional bool mergeable = 5;
 
   enum CompactionState {
     NONE = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 02b0d2c..889f548 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -365,6 +365,7 @@ message RunCatalogScanRequest {
 }
 
 message RunCatalogScanResponse {
+  // This is how many archiving tasks we started as a result of this scan.
   optional int32 scan_result = 1;
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 6b7206f..f0668d8 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -370,3 +370,25 @@ message MoveRegionStateData {
   required ServerName source_server = 2;
   required ServerName destination_server = 3;
 }
+
+enum GCRegionState {
+  GC_REGION_PREPARE = 1;
+  GC_REGION_ARCHIVE = 2;
+  GC_REGION_PURGE_METADATA = 3;
+}
+
+message GCRegionStateData {
+  required RegionInfo region_info = 1;
+}
+
+enum GCMergedRegionsState {
+  GC_MERGED_REGIONS_PREPARE = 1;
+  GC_MERGED_REGIONS_PURGE = 2;
+  GC_REGION_EDIT_METADATA = 3;
+}
+
+message GCMergedRegionsStateData {
+  required RegionInfo parent_a = 1;
+  required RegionInfo parent_b = 2;
+  required RegionInfo merged_child = 3;
+}

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index 6e727f6..edd163c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -27,7 +27,6 @@ import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
@@ -39,12 +38,14 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.favored.FavoredNodesManager;
+import org.apache.hadoop.hbase.master.assignment.GCMergedRegionsProcedure;
+import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -192,8 +193,6 @@ public class CatalogJanitor extends ScheduledChore {
    * If merged region no longer holds reference to the merge regions, archive
    * merge region on hdfs and perform deleting references in hbase:meta
    * @param mergedRegion
-   * @param regionA
-   * @param regionB
    * @return true if we delete references in merged region on hbase:meta and archive
    *         the files on the file system
    * @throws IOException
@@ -215,15 +214,11 @@ public class CatalogJanitor extends ScheduledChore {
       LOG.debug("Deleting region " + regionA.getRegionNameAsString() + " and "
           + regionB.getRegionNameAsString()
           + " from fs because merged region no longer holds references");
-      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionA);
-      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, regionB);
-      MetaTableAccessor.deleteMergeQualifiers(services.getConnection(), mergedRegion);
-      services.getServerManager().removeRegion(regionA);
-      services.getServerManager().removeRegion(regionB);
-      FavoredNodesManager fnm = this.services.getFavoredNodesManager();
-      if (fnm != null) {
-        fnm.deleteFavoredNodesForRegions(Lists.newArrayList(regionA, regionB));
-      }
+      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
+      GCMergedRegionsProcedure proc =
+          new GCMergedRegionsProcedure(pe.getEnvironment(),mergedRegion,  regionA, regionB);
+      proc.setOwner(pe.getEnvironment().getRequestUser().getShortName());
+      pe.submitProcedure(proc);
       return true;
     }
     return false;
@@ -232,22 +227,21 @@ public class CatalogJanitor extends ScheduledChore {
   /**
    * Run janitorial scan of catalog <code>hbase:meta</code> table looking for
    * garbage to collect.
-   * @return number of cleaned regions
+   * @return number of archiving jobs started.
    * @throws IOException
    */
   int scan() throws IOException {
+    int result = 0;
     try {
       if (!alreadyRunning.compareAndSet(false, true)) {
         LOG.debug("CatalogJanitor already running");
-        return 0;
+        return result;
       }
       Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> scanTriple =
         getMergedRegionsAndSplitParents();
-      int count = scanTriple.getFirst();
       /**
        * clean merge regions first
        */
-      int mergeCleaned = 0;
       Map<HRegionInfo, Result> mergedRegions = scanTriple.getSecond();
       for (Map.Entry<HRegionInfo, Result> e : mergedRegions.entrySet()) {
         if (this.services.isInMaintenanceMode()) {
@@ -266,7 +260,7 @@ public class CatalogJanitor extends ScheduledChore {
               + " in merged region " + e.getKey().getRegionNameAsString());
         } else {
           if (cleanMergeRegion(e.getKey(), regionA, regionB)) {
-            mergeCleaned++;
+            result++;
           }
         }
       }
@@ -276,7 +270,6 @@ public class CatalogJanitor extends ScheduledChore {
       Map<HRegionInfo, Result> splitParents = scanTriple.getThird();
 
       // Now work on our list of found parents. See if any we can clean up.
-      int splitCleaned = 0;
       // regions whose parents are still around
       HashSet<String> parentNotCleaned = new HashSet<>();
       for (Map.Entry<HRegionInfo, Result> e : splitParents.entrySet()) {
@@ -286,8 +279,8 @@ public class CatalogJanitor extends ScheduledChore {
         }
 
         if (!parentNotCleaned.contains(e.getKey().getEncodedName()) &&
-            cleanParent(e.getKey(), e.getValue())) {
-          splitCleaned++;
+              cleanParent(e.getKey(), e.getValue())) {
+            result++;
         } else {
           // We could not clean the parent, so it's daughters should not be
           // cleaned either (HBASE-6160)
@@ -297,16 +290,7 @@ public class CatalogJanitor extends ScheduledChore {
           parentNotCleaned.add(daughters.getSecond().getEncodedName());
         }
       }
-      if ((mergeCleaned + splitCleaned) != 0) {
-        LOG.info("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
-            + " unreferenced merged region(s) and " + splitCleaned
-            + " unreferenced parent region(s)");
-      } else if (LOG.isTraceEnabled()) {
-        LOG.trace("Scanned " + count + " catalog row(s), gc'd " + mergeCleaned
-            + " unreferenced merged region(s) and " + splitCleaned
-            + " unreferenced parent region(s)");
-      }
-      return mergeCleaned + splitCleaned;
+      return result;
     } finally {
       alreadyRunning.set(false);
     }
@@ -348,39 +332,28 @@ public class CatalogJanitor extends ScheduledChore {
    */
   boolean cleanParent(final HRegionInfo parent, Result rowContent)
   throws IOException {
-    boolean result = false;
     // Check whether it is a merged region and not clean reference
     // No necessary to check MERGEB_QUALIFIER because these two qualifiers will
     // be inserted/deleted together
     if (rowContent.getValue(HConstants.CATALOG_FAMILY, HConstants.MERGEA_QUALIFIER) != null) {
       // wait cleaning merge region first
-      return result;
+      return false;
     }
     // Run checks on each daughter split.
     PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(rowContent);
     Pair<Boolean, Boolean> a = checkDaughterInFs(parent, daughters.getFirst());
     Pair<Boolean, Boolean> b = checkDaughterInFs(parent, daughters.getSecond());
     if (hasNoReferences(a) && hasNoReferences(b)) {
-      LOG.debug("Deleting region " + parent.getRegionNameAsString() +
-        " because daughter splits no longer hold references");
-      FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
-      if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent);
-      HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
-      AssignmentManager am = this.services.getAssignmentManager();
-      if (am != null) {
-        if (am.getRegionStates() != null) {
-          am.getRegionStates().deleteRegion(parent);
-        }
-      }
-      MetaTableAccessor.deleteRegion(this.connection, parent);
-      services.getServerManager().removeRegion(parent);
-      FavoredNodesManager fnm = this.services.getFavoredNodesManager();
-      if (fnm != null) {
-        fnm.deleteFavoredNodesForRegions(Lists.newArrayList(parent));
-      }
-      result = true;
+      LOG.debug("Deleting region " + parent.getShortNameToLog() +
+        " because daughters -- " + daughters.getFirst() + ", " + daughters.getSecond() +
+        " -- no longer hold references");
+      ProcedureExecutor<MasterProcedureEnv> pe = this.services.getMasterProcedureExecutor();
+      GCRegionProcedure proc = new GCRegionProcedure(pe.getEnvironment(), parent);
+      proc.setOwner(pe.getEnvironment().getRequestUser().getShortName());
+      pe.submitProcedure(new GCRegionProcedure(pe.getEnvironment(), parent));
+      return true;
     }
-    return result;
+    return false;
   }
 
   /**

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 158155e..36f6f08 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
@@ -71,6 +71,15 @@ public class AssignProcedure extends RegionTransitionProcedure {
 
   private boolean forceNewPlan = false;
 
+  /**
+   * Gets set as desired target on move, merge, etc., when we want to go to a particular server.
+   * We may not be able to respect this request but will try. When it is NOT set, then we ask
+   * the balancer to assign. This value is used below in startTransition to set regionLocation if
+   * non-null. Setting regionLocation in regionServerNode is how we override balancer setting
+   * destination.
+   */
+  protected volatile ServerName targetServer;
+
   public AssignProcedure() {
     // Required by the Procedure framework to create the procedure on replay
     super();
@@ -83,22 +92,18 @@ public class AssignProcedure extends RegionTransitionProcedure {
   public AssignProcedure(final HRegionInfo regionInfo, final boolean forceNewPlan) {
     super(regionInfo);
     this.forceNewPlan = forceNewPlan;
-    this.server = null;
+    this.targetServer = null;
   }
 
   public AssignProcedure(final HRegionInfo regionInfo, final ServerName destinationServer) {
     super(regionInfo);
     this.forceNewPlan = false;
-    this.server = destinationServer;
-  }
-
-  public ServerName getServer() {
-    return this.server;
+    this.targetServer = destinationServer;
   }
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.ASSIGN;
+    return TableOperationType.REGION_ASSIGN;
   }
 
   @Override
@@ -119,8 +124,8 @@ public class AssignProcedure extends RegionTransitionProcedure {
     if (forceNewPlan) {
       state.setForceNewPlan(true);
     }
-    if (server != null) {
-      state.setTargetServer(ProtobufUtil.toServerName(server));
+    if (this.targetServer != null) {
+      state.setTargetServer(ProtobufUtil.toServerName(this.targetServer));
     }
     state.build().writeDelimitedTo(stream);
   }
@@ -132,7 +137,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
     setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
     forceNewPlan = state.getForceNewPlan();
     if (state.hasTargetServer()) {
-      server = ProtobufUtil.toServerName(state.getTargetServer());
+      this.targetServer = ProtobufUtil.toServerName(state.getTargetServer());
     }
   }
 
@@ -146,8 +151,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
     }
     // If the region is SPLIT, we can't assign it.
     if (regionNode.isInState(State.SPLIT)) {
-      LOG.info("SPLIT, cannot be assigned; " +
-          this + "; " + regionNode.toShortString());
+      LOG.info("SPLIT, cannot be assigned; " + this + "; " + regionNode.toShortString());
       return false;
     }
 
@@ -163,16 +167,22 @@ public class AssignProcedure extends RegionTransitionProcedure {
       return false;
     }
 
-    // send assign (add into assign-pool). region is now in OFFLINE state
+    // Send assign (add into assign-pool). Region is now in OFFLINE state. Setting offline state
+    // scrubs what was the old region location. Setting a new regionLocation here is how we retain
+    // old assignment or specify target server if a move or merge. See
+    // AssignmentManager#processAssignQueue. Otherwise, balancer gives us location.
     ServerName lastRegionLocation = regionNode.offline();
     boolean retain = false;
     if (!forceNewPlan) {
-      if (this.server != null) {
-        regionNode.setRegionLocation(server);
+      if (this.targetServer != null) {
+        retain = targetServer.equals(lastRegionLocation);
+        regionNode.setRegionLocation(targetServer);
       } else {
-        // Try to 'retain' old assignment.
-        retain = true;
-        if (lastRegionLocation != null) regionNode.setRegionLocation(lastRegionLocation);
+        if (lastRegionLocation != null) {
+          // Try and keep the location we had before we offlined.
+          retain = true;
+          regionNode.setRegionLocation(lastRegionLocation);
+        }
       }
     }
     LOG.info("Start " + this + "; " + regionNode.toShortString() +
@@ -193,13 +203,6 @@ public class AssignProcedure extends RegionTransitionProcedure {
     if (regionNode.getRegionLocation() == null) {
       setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
       return true;
-    } else if (this.server == null) {
-      // Update our server reference target to align with regionNode regionLocation
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Setting tgt=" + regionNode.getRegionLocation() +
-          " from regionStateNode.getRegionLocation " + this + "; " + regionNode.toShortString());
-      }
-      this.server = regionNode.getRegionLocation();
     }
 
     if (!isServerOnline(env, regionNode)) {
@@ -288,7 +291,7 @@ public class AssignProcedure extends RegionTransitionProcedure {
       aborted.set(true);
     }
     this.forceNewPlan = true;
-    this.server = null;
+    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.
@@ -318,4 +321,10 @@ public class AssignProcedure extends RegionTransitionProcedure {
       final IOException exception) {
     handleFailure(env, regionNode);
   }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    super.toStringClassDetails(sb);
+    if (this.targetServer != null) sb.append(", target=").append(this.targetServer);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
new file mode 100644
index 0000000..c7d97ee
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCMergedRegionsProcedure.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsState;
+
+/**
+ * GC regions that have been Merged.
+ * Caller determines if it is GC time. This Procedure does not check.
+ * <p>This is a Table Procedure. We take a read lock on the Table.
+ * We do NOT keep a lock for the life of this procedure. The subprocedures
+ * take locks on the Regions they are purging.
+ */
+@InterfaceAudience.Private
+public class GCMergedRegionsProcedure
+extends AbstractStateMachineTableProcedure<GCMergedRegionsState> {
+  private static final Log LOG = LogFactory.getLog(GCMergedRegionsProcedure.class);
+  private HRegionInfo father;
+  private HRegionInfo mother;
+  private HRegionInfo mergedChild;
+
+  public GCMergedRegionsProcedure(final MasterProcedureEnv env,
+      final HRegionInfo mergedChild,
+      final HRegionInfo father,
+      final HRegionInfo mother) {
+    super(env);
+    this.father = father;
+    this.mother = mother;
+    this.mergedChild = mergedChild;
+  }
+
+  public GCMergedRegionsProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.MERGED_REGIONS_GC;
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, GCMergedRegionsState state)
+  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    try {
+      switch (state) {
+      case GC_MERGED_REGIONS_PREPARE:
+        // Nothing to do to prepare.
+        setNextState(GCMergedRegionsState.GC_MERGED_REGIONS_PURGE);
+        break;
+      case GC_MERGED_REGIONS_PURGE:
+        addChildProcedure(createGCRegionProcedures(env));
+        setNextState(GCMergedRegionsState.GC_REGION_EDIT_METADATA);
+        break;
+      case GC_REGION_EDIT_METADATA:
+        MetaTableAccessor.deleteMergeQualifiers(env.getMasterServices().getConnection(), mergedChild);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException ioe) {
+      // TODO: This is going to spew log?
+      LOG.warn("Error trying to GC merged regions " + this.father.getShortNameToLog() +
+          " & " + this.mother.getShortNameToLog() + "; retrying...", ioe);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  private GCRegionProcedure[] createGCRegionProcedures(final MasterProcedureEnv env) {
+    GCRegionProcedure [] procs = new GCRegionProcedure[2];
+    int index = 0;
+    for (HRegionInfo hri: new HRegionInfo [] {this.father, this.mother}) {
+      GCRegionProcedure proc = new GCRegionProcedure(env, hri);
+      proc.setOwner(env.getRequestUser().getShortName());
+      procs[index++] = proc;
+    }
+    return procs;
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, GCMergedRegionsState state)
+  throws IOException, InterruptedException {
+    // no-op
+  }
+
+  @Override
+  protected GCMergedRegionsState getState(int stateId) {
+    return GCMergedRegionsState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(GCMergedRegionsState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected GCMergedRegionsState getInitialState() {
+    return GCMergedRegionsState.GC_MERGED_REGIONS_PREPARE;
+  }
+
+  @Override
+  protected void serializeStateData(OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+    final MasterProcedureProtos.GCMergedRegionsStateData.Builder msg =
+        MasterProcedureProtos.GCMergedRegionsStateData.newBuilder().
+        setParentA(HRegionInfo.convert(this.father)).
+        setParentB(HRegionInfo.convert(this.mother)).
+        setMergedChild(HRegionInfo.convert(this.mergedChild));
+    msg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  protected void deserializeStateData(InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+    final MasterProcedureProtos.GCMergedRegionsStateData msg =
+        MasterProcedureProtos.GCMergedRegionsStateData.parseDelimitedFrom(stream);
+    this.father = HRegionInfo.convert(msg.getParentA());
+    this.mother = HRegionInfo.convert(msg.getParentB());
+    this.mergedChild = HRegionInfo.convert(msg.getMergedChild());
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" child=");
+    sb.append(this.mergedChild.getShortNameToLog());
+    sb.append(", father=");
+    sb.append(this.father.getShortNameToLog());
+    sb.append(", mother=");
+    sb.append(this.mother.getShortNameToLog());
+  }
+
+  @Override
+  public TableName getTableName() {
+    return this.mergedChild.getTable();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
new file mode 100644
index 0000000..05766f7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/GCRegionProcedure.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.favored.FavoredNodesManager;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState;
+
+import com.google.common.collect.Lists;
+
+/**
+ * GC a Region that is no longer in use. It has been split or merged away.
+ * Caller determines if it is GC time. This Procedure does not check.
+ * <p>This is a Region StateMachine Procedure. We take a read lock on the Table and then
+ * exclusive on the Region.
+ */
+@InterfaceAudience.Private
+public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCRegionState> {
+  private static final Log LOG = LogFactory.getLog(GCRegionProcedure.class);
+
+  public GCRegionProcedure(final MasterProcedureEnv env, final HRegionInfo hri) {
+    super(env, hri);
+  }
+
+  public GCRegionProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REGION_GC;
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, GCRegionState state)
+  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    MasterServices masterServices = env.getMasterServices();
+    try {
+      switch (state) {
+      case GC_REGION_PREPARE:
+        // Nothing to do to prepare.
+        setNextState(GCRegionState.GC_REGION_ARCHIVE);
+        break;
+      case GC_REGION_ARCHIVE:
+        FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
+        if (HFileArchiver.exists(masterServices.getConfiguration(), fs, getRegion())) {
+          if (LOG.isDebugEnabled()) LOG.debug("Archiving region=" + getRegion().getShortNameToLog());
+          HFileArchiver.archiveRegion(masterServices.getConfiguration(), fs, getRegion());
+        }
+        setNextState(GCRegionState.GC_REGION_PURGE_METADATA);
+        break;
+      case GC_REGION_PURGE_METADATA:
+        // TODO: Purge metadata before removing from HDFS? This ordering is copied
+        // from CatalogJanitor.
+        AssignmentManager am = masterServices.getAssignmentManager();
+        if (am != null) {
+          if (am.getRegionStates() != null) {
+            am.getRegionStates().deleteRegion(getRegion());
+          }
+        }
+        MetaTableAccessor.deleteRegion(masterServices.getConnection(), getRegion());
+        masterServices.getServerManager().removeRegion(getRegion());
+        FavoredNodesManager fnm = masterServices.getFavoredNodesManager();
+        if (fnm != null) {
+          fnm.deleteFavoredNodesForRegions(Lists.newArrayList(getRegion()));
+        }
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException ioe) {
+      // TODO: This is going to spew log?
+      LOG.warn("Error trying to GC " + getRegion().getShortNameToLog() + "; retrying...", ioe);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, GCRegionState state) throws IOException, InterruptedException {
+    // no-op
+  }
+
+  @Override
+  protected GCRegionState getState(int stateId) {
+    return GCRegionState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(GCRegionState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected GCRegionState getInitialState() {
+    return GCRegionState.GC_REGION_PREPARE;
+  }
+
+  @Override
+  protected void serializeStateData(OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+    final MasterProcedureProtos.GCRegionStateData.Builder msg =
+        MasterProcedureProtos.GCRegionStateData.newBuilder()
+        .setRegionInfo(HRegionInfo.convert(getRegion()));
+    msg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  protected void deserializeStateData(InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+    final MasterProcedureProtos.GCRegionStateData msg =
+        MasterProcedureProtos.GCRegionStateData.parseDelimitedFrom(stream);
+    setRegion(HRegionInfo.convert(msg.getRegionInfo()));
+  }
+
+  @Override
+  protected org.apache.hadoop.hbase.procedure2.Procedure.LockState acquireLock(MasterProcedureEnv env) {
+    return super.acquireLock(env);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 177f397..2b1de9d 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
@@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.master.assignment;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.Arrays;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
@@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.CatalogJanitor;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
@@ -53,29 +52,33 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
-import org.apache.hadoop.hbase.procedure2.Procedure.LockState;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MergeTableRegionsState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.lmax.disruptor.YieldingWaitStrategy;
+
 /**
  * The procedure to Merge a region in a table.
+ * This procedure takes an exclusive table lock since it is working over multiple regions.
+ * It holds the lock for the life of the procedure.
  */
 @InterfaceAudience.Private
 public class MergeTableRegionsProcedure
     extends AbstractStateMachineTableProcedure<MergeTableRegionsState> {
   private static final Log LOG = LogFactory.getLog(MergeTableRegionsProcedure.class);
-
   private Boolean traceEnabled;
-
+  private volatile boolean lock = false;
   private ServerName regionLocation;
-  private String regionsToMergeListFullName;
-
   private HRegionInfo[] regionsToMerge;
   private HRegionInfo mergedRegion;
   private boolean forcible;
@@ -112,8 +115,6 @@ public class MergeTableRegionsProcedure
     this.regionsToMerge = regionsToMerge;
     this.mergedRegion = createMergedRegionInfo(regionsToMerge);
     this.forcible = forcible;
-
-    this.regionsToMergeListFullName = getRegionsToMergeListFullNameString();
   }
 
   private static void checkRegionsToMerge(final HRegionInfo[] regionsToMerge,
@@ -198,7 +199,8 @@ public class MergeTableRegionsProcedure
   @Override
   protected Flow executeFromState(
       final MasterProcedureEnv env,
-      final MergeTableRegionsState state) throws InterruptedException {
+      final MergeTableRegionsState state)
+  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     if (LOG.isDebugEnabled()) {
       LOG.debug(this + " execute state=" + state);
     }
@@ -250,7 +252,7 @@ public class MergeTableRegionsProcedure
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
-      LOG.warn("Error trying to merge regions " + getRegionsToMergeListFullNameString() +
+      LOG.warn("Error trying to merge regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
         " in the table " + getTableName() + " (in state=" + state + ")", e);
 
       setFailure("master-merge-regions", e);
@@ -303,7 +305,7 @@ public class MergeTableRegionsProcedure
       // This will be retried. Unless there is a bug in the code,
       // this should be just a "temporary error" (e.g. network down)
       LOG.warn("Failed rollback attempt step " + state + " for merging the regions "
-          + getRegionsToMergeListFullNameString() + " in table " + getTableName(), e);
+          + HRegionInfo.getShortNameToLog(regionsToMerge) + " in table " + getTableName(), e);
       throw e;
     }
   }
@@ -379,7 +381,7 @@ public class MergeTableRegionsProcedure
     sb.append(" table=");
     sb.append(getTableName());
     sb.append(", regions=");
-    sb.append(getRegionsToMergeListFullNameString());
+    sb.append(HRegionInfo.getShortNameToLog(regionsToMerge));
     sb.append(", forcibly=");
     sb.append(forcible);
   }
@@ -397,23 +399,35 @@ public class MergeTableRegionsProcedure
       }
       return LockState.LOCK_EVENT_WAIT;
     }
+    this.lock = true;
     return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(final MasterProcedureEnv env) {
+    this.lock = false;
     env.getProcedureScheduler().wakeRegions(this, getTableName(),
       mergedRegion, regionsToMerge[0], regionsToMerge[1]);
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  @Override
+  protected boolean hasLock(MasterProcedureEnv env) {
+    return this.lock;
+  }
+
+  @Override
   public TableName getTableName() {
     return mergedRegion.getTable();
   }
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.MERGE;
+    return TableOperationType.REGION_MERGE;
   }
 
   /**
@@ -429,8 +443,8 @@ public class MergeTableRegionsProcedure
     boolean regionAHasMergeQualifier = !catalogJanitor.cleanMergeQualifier(regionsToMerge[0]);
     if (regionAHasMergeQualifier
         || !catalogJanitor.cleanMergeQualifier(regionsToMerge[1])) {
-      String msg = "Skip merging regions " + getRegionsToMergeListFullNameString()
-        + ", because region "
+      String msg = "Skip merging regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+        ", because region "
         + (regionAHasMergeQualifier ? regionsToMerge[0].getEncodedName() : regionsToMerge[1]
               .getEncodedName()) + " has merge qualifier";
       LOG.warn(msg);
@@ -458,9 +472,43 @@ public class MergeTableRegionsProcedure
           new IOException("Merge of " + regionsStr + " failed because merge switch is off"));
       return false;
     }
+    
+
+    // Ask the remote regionserver if regions are mergeable. If we get an IOE, report it
+    // along w/ the failure so can see why we are not mergeable at this time.
+    IOException mergeableCheckIOE = null;
+    boolean mergeable = false;
+    RegionState current = regionStateA;
+    try {
+      mergeable = isMergeable(env, current);
+    } catch (IOException e) {
+      mergeableCheckIOE = e;
+    }
+    if (mergeable && mergeableCheckIOE == null) {
+      current = regionStateB;
+      try {
+        mergeable = isMergeable(env, current);
+      } catch (IOException e) {
+        mergeableCheckIOE = e;
+      }
+    }
+    if (!mergeable) {
+      IOException e = new IOException(current.getRegion().getShortNameToLog() + " NOT mergeable");
+      if (mergeableCheckIOE != null) e.initCause(mergeableCheckIOE);
+      super.setFailure(getClass().getSimpleName(), e);
+      return false;
+    }
+
     return true;
   }
 
+  private boolean isMergeable(final MasterProcedureEnv env, final RegionState rs)
+  throws IOException {
+    GetRegionInfoResponse response =
+      Util.getRegionInfoResponse(env, rs.getServerName(), rs.getRegion());
+    return response.hasSplittable() && response.getSplittable();
+  }
+
   /**
    * Pre merge region action
    * @param env MasterProcedureEnv
@@ -471,7 +519,8 @@ public class MergeTableRegionsProcedure
       boolean ret = cpHost.preMergeRegionsAction(regionsToMerge, getUser());
       if (ret) {
         throw new IOException(
-          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+          "Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+          " merge.");
       }
     }
     // TODO: Clean up split and merge. Currently all over the place.
@@ -640,7 +689,8 @@ public class MergeTableRegionsProcedure
 
       if (ret) {
         throw new IOException(
-          "Coprocessor bypassing regions " + getRegionsToMergeListFullNameString() + " merge.");
+          "Coprocessor bypassing regions " + HRegionInfo.getShortNameToLog(regionsToMerge) +
+          " merge.");
       }
       try {
         for (Mutation p : metaEntries) {
@@ -656,10 +706,9 @@ public class MergeTableRegionsProcedure
 
   /**
    * Add merged region to META and delete original regions.
-   * @param env MasterProcedureEnv
-   * @throws IOException
    */
-  private void updateMetaForMergedRegions(final MasterProcedureEnv env) throws IOException {
+  private void updateMetaForMergedRegions(final MasterProcedureEnv env)
+  throws IOException, ProcedureYieldException {
     final ServerName serverName = getServerName(env);
     env.getAssignmentManager().markRegionAsMerged(mergedRegion, serverName,
       regionsToMerge[0], regionsToMerge[1]);
@@ -695,8 +744,12 @@ public class MergeTableRegionsProcedure
    */
   private ServerName getServerName(final MasterProcedureEnv env) {
     if (regionLocation == null) {
-      regionLocation = env.getAssignmentManager().getRegionStates()
-        .getRegionServerOfRegion(regionsToMerge[0]);
+      regionLocation = env.getAssignmentManager().getRegionStates().
+          getRegionServerOfRegion(regionsToMerge[0]);
+      // May still be null here but return null and let caller deal.
+      // Means we lost the in-memory-only location. We are in recovery
+      // or so. The caller should be able to deal w/ a null ServerName.
+      // Let them go to the Balancer to find one to use instead.
     }
     return regionLocation;
   }
@@ -704,28 +757,6 @@ public class MergeTableRegionsProcedure
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.
-   * @param fullName whether return only encoded name
-   * @return region names in a list
-   */
-  private String getRegionsToMergeListFullNameString() {
-    if (regionsToMergeListFullName == null) {
-      final StringBuilder sb = new StringBuilder("[");
-      int i = 0;
-      while(i < regionsToMerge.length - 1) {
-        sb.append(regionsToMerge[i].getRegionNameAsString());
-        sb.append(", ");
-        i++;
-      }
-      sb.append(regionsToMerge[i].getRegionNameAsString());
-      sb.append("]");
-      regionsToMergeListFullName = sb.toString();
-    }
-    return regionsToMergeListFullName;
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
    * @return traceEnabled
    */
   private Boolean isTraceEnabled() {
@@ -734,4 +765,12 @@ public class MergeTableRegionsProcedure
     }
     return traceEnabled;
   }
-}
+
+  /**
+   * @return The merged region. Maybe be null if called to early or we failed.
+   */
+  @VisibleForTesting
+  public HRegionInfo getMergedRegion() {
+    return this.mergedRegion;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 b1445fb..6cc04e4 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
@@ -29,10 +29,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
 import org.apache.hadoop.hbase.master.RegionPlan;
-import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
@@ -40,14 +39,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.M
 /**
  * Procedure that implements a RegionPlan.
  * It first runs an unassign subprocedure followed
- * by an assign subprocedure.
+ * by an assign subprocedure. It takes a lock on the region being moved.
+ * It holds the lock for the life of the procedure.
  */
 @InterfaceAudience.Private
-public class MoveRegionProcedure
-    extends StateMachineProcedure<MasterProcedureEnv, MoveRegionState>
-    implements TableProcedureInterface {
+public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<MoveRegionState> {
   private static final Log LOG = LogFactory.getLog(MoveRegionProcedure.class);
-
   private RegionPlan plan;
 
   public MoveRegionProcedure() {
@@ -61,15 +58,6 @@ public class MoveRegionProcedure
   }
 
   @Override
-  protected boolean holdLock(MasterProcedureEnv env) {
-    // Hold the lock for the duration of the move otherwise something like
-    // a call to split might come in when we do not hold the lock; i.e.
-    // at the point between completion of unassign and before we do the
-    // assign step (I've seen it in test).
-    return true;
-  }
-
-  @Override
   protected Flow executeFromState(final MasterProcedureEnv env, final MoveRegionState state)
       throws InterruptedException {
     if (LOG.isTraceEnabled()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 3e52780..21e0d9c 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
@@ -125,7 +125,7 @@ public class RegionStateStore {
       final long openSeqNum = -1;
 
       // TODO: move under trace, now is visible for debugging
-      LOG.info(String.format("Load hbase:meta entry region=%s state=%s lastHost=%s regionLocation=%s",
+      LOG.info(String.format("Load hbase:meta entry region=%s regionState=%s lastHost=%s regionLocation=%s",
         regionInfo, state, lastHost, regionLocation));
 
       visitor.visitRegionState(regionInfo, state, regionLocation, lastHost, openSeqNum);
@@ -167,19 +167,19 @@ public class RegionStateStore {
     final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo));
     MetaTableAccessor.addRegionInfo(put, regionInfo);
     final StringBuilder info = new StringBuilder("pid=" + pid + " updating hbase:meta row=");
-    info.append(regionInfo.getRegionNameAsString()).append(" with state=").append(state);
+    info.append(regionInfo.getRegionNameAsString()).append(", regionState=").append(state);
     if (openSeqNum >= 0) {
       Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
           "Open region should be on a server");
       MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, -1, replicaId);
       info.append(", openSeqNum=").append(openSeqNum);
-      info.append(", location=").append(regionLocation);
+      info.append(", regionLocation=").append(regionLocation);
     } else if (regionLocation != null && !regionLocation.equals(lastHost)) {
       // Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients
       // currently; they want a server to hit. TODO: Make clients wait if no location.
       put.addImmutable(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId),
           Bytes.toBytes(regionLocation.getServerName()));
-      info.append(", sn=").append(regionLocation);
+      info.append(", regionLocation=").append(regionLocation);
     }
     put.addImmutable(HConstants.CATALOG_FAMILY, getStateColumn(replicaId),
       Bytes.toBytes(state.name()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 2a3b72f..082e171 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
@@ -103,6 +103,11 @@ public class RegionStates {
     private volatile RegionTransitionProcedure procedure = null;
     private volatile ServerName regionLocation = null;
     private volatile ServerName lastHost = null;
+    /**
+     * A Region-in-Transition (RIT) moves through states.
+     * See {@link State} for complete list. A Region that
+     * is opened moves from OFFLINE => OPENING => OPENED.
+     */
     private volatile State state = State.OFFLINE;
 
     /**
@@ -183,8 +188,8 @@ public class RegionStates {
     
     public ServerName setRegionLocation(final ServerName serverName) {
       ServerName lastRegionLocation = this.regionLocation;
-      if (serverName == null) {
-        LOG.debug("REMOVE tracking when we are set to null " + this, new Throwable("DEBUG"));
+      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();
@@ -274,7 +279,8 @@ public class RegionStates {
     }
  
     public String toShortString() {
-      return String.format("regionState=%s, regionLocation=%s", getState(), getRegionLocation());
+      // rit= is the current Region-In-Transition State -- see State enum.
+      return String.format("rit=%s, location=%s", getState(), getRegionLocation());
     }
 
     public String toDescriptiveString() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 5f19bdc..6dc809b 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
@@ -67,8 +67,6 @@ public abstract class RegionTransitionProcedure
       RegionTransitionState.REGION_TRANSITION_QUEUE;
   private HRegionInfo regionInfo;
   private volatile boolean lock = false;
-  // Server we assign or unassign from -- the target.
-  protected volatile ServerName server;
 
   public RegionTransitionProcedure() {
     // Required by the Procedure framework to create the procedure on replay
@@ -105,8 +103,6 @@ public abstract class RegionTransitionProcedure
     sb.append(getTableName());
     sb.append(", region=");
     sb.append(getRegionInfo() == null? null: getRegionInfo().getEncodedName());
-    sb.append(", tgt=");
-    sb.append(getServer());
   }
 
   public RegionStateNode getRegionState(final MasterProcedureEnv env) {
@@ -234,12 +230,18 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
+  protected void toStringState(StringBuilder builder) {
+    super.toStringState(builder);
+    RegionTransitionState ts = this.transitionState;
+    if (!isFinished() && ts != null) {
+      builder.append(":").append(ts);
+    }
+  }
+
+  @Override
   protected Procedure[] execute(final MasterProcedureEnv env) throws ProcedureSuspendedException {
     final AssignmentManager am = env.getAssignmentManager();
     final RegionStateNode regionNode = getRegionState(env);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("" + transitionState + " " + this + "; " + regionNode.toShortString());
-    }
     if (!am.addRegionInTransition(regionNode, this)) {
       String msg = String.format(
         "There is already another procedure running on this region this=%s owner=%s",
@@ -262,6 +264,7 @@ public abstract class RegionTransitionProcedure
             }
             transitionState = RegionTransitionState.REGION_TRANSITION_DISPATCH;
             if (env.getProcedureScheduler().waitEvent(regionNode.getProcedureEvent(), this)) {
+              // Why this suspend? Because we want to ensure Store happens before proceed?
               throw new ProcedureSuspendedException();
             }
             break;
@@ -369,8 +372,4 @@ public abstract class RegionTransitionProcedure
     // the client does not know about this procedure.
     return false;
   }
-
-  public ServerName getServer() {
-    return this.server;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 1903a1d..4ed1931 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
@@ -42,25 +42,26 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
-import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -73,17 +74,14 @@ import com.google.common.annotations.VisibleForTesting;
 
 /**
  * The procedure to split a region in a table.
+ * Takes lock on the parent region.
+ * It holds the lock for the life of the procedure.
  */
 @InterfaceAudience.Private
 public class SplitTableRegionProcedure
-    extends AbstractStateMachineTableProcedure<SplitTableRegionState> {
+    extends AbstractStateMachineRegionProcedure<SplitTableRegionState> {
   private static final Log LOG = LogFactory.getLog(SplitTableRegionProcedure.class);
-
   private Boolean traceEnabled = null;
-
-  private volatile boolean lock = false;
-
-  private HRegionInfo parentHRI;
   private HRegionInfo daughter_1_HRI;
   private HRegionInfo daughter_2_HRI;
 
@@ -93,22 +91,16 @@ public class SplitTableRegionProcedure
 
   public SplitTableRegionProcedure(final MasterProcedureEnv env,
       final HRegionInfo regionToSplit, final byte[] splitRow) throws IOException {
-    super(env);
+    super(env, regionToSplit);
 
     checkSplitRow(regionToSplit, splitRow);
 
-    this.parentHRI = regionToSplit;
-
     final TableName table = regionToSplit.getTable();
     final long rid = getDaughterRegionIdTimestamp(regionToSplit);
     this.daughter_1_HRI = new HRegionInfo(table, regionToSplit.getStartKey(), splitRow, false, rid);
     this.daughter_2_HRI = new HRegionInfo(table, splitRow, regionToSplit.getEndKey(), false, rid);
   }
 
-  protected void setFailure(Throwable cause) {
-    super.setFailure(getClass().getSimpleName(), cause);
-  }
-
   private static void checkSplitRow(final HRegionInfo regionToSplit, final byte[] splitRow)
       throws IOException {
     if (splitRow == null || splitRow.length == 0) {
@@ -196,7 +188,7 @@ public class SplitTableRegionProcedure
         throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
-      String msg = "Error trying to split region " + parentHRI.getEncodedName() + " in the table "
+      String msg = "Error trying to split region " + getParentRegion().getEncodedName() + " in the table "
           + getTableName() + " (in state=" + state + ")";
       if (!isRollbackSupported(state)) {
         // We reach a state that cannot be rolled back. We just need to keep retry.
@@ -245,7 +237,7 @@ public class SplitTableRegionProcedure
       // this should be just a "temporary error" (e.g. network down)
       LOG.warn("pid=" + getProcId() + " failed rollback attempt step " + state +
           " for splitting the region "
-        + parentHRI.getEncodedName() + " in table " + getTableName(), e);
+        + getParentRegion().getEncodedName() + " in table " + getTableName(), e);
       throw e;
     }
   }
@@ -290,7 +282,7 @@ public class SplitTableRegionProcedure
     final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg =
         MasterProcedureProtos.SplitTableRegionStateData.newBuilder()
         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
-        .setParentRegionInfo(HRegionInfo.convert(parentHRI))
+        .setParentRegionInfo(HRegionInfo.convert(getRegion()))
         .addChildRegionInfo(HRegionInfo.convert(daughter_1_HRI))
         .addChildRegionInfo(HRegionInfo.convert(daughter_2_HRI));
     splitTableRegionMsg.build().writeDelimitedTo(stream);
@@ -303,7 +295,7 @@ public class SplitTableRegionProcedure
     final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg =
         MasterProcedureProtos.SplitTableRegionStateData.parseDelimitedFrom(stream);
     setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo()));
-    parentHRI = HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo());
+    setRegion(HRegionInfo.convert(splitTableRegionsMsg.getParentRegionInfo()));
     assert(splitTableRegionsMsg.getChildRegionInfoCount() == 2);
     daughter_1_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(0));
     daughter_2_HRI = HRegionInfo.convert(splitTableRegionsMsg.getChildRegionInfo(1));
@@ -315,45 +307,20 @@ public class SplitTableRegionProcedure
     sb.append(" table=");
     sb.append(getTableName());
     sb.append(", parent=");
-    sb.append(parentHRI.getShortNameToLog());
+    sb.append(getParentRegion().getShortNameToLog());
     sb.append(", daughterA=");
     sb.append(daughter_1_HRI.getShortNameToLog());
     sb.append(", daughterB=");
     sb.append(daughter_2_HRI.getShortNameToLog());
   }
 
-  @Override
-  protected LockState acquireLock(final MasterProcedureEnv env) {
-    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
-
-    if (env.getProcedureScheduler().waitRegions(this, getTableName(), parentHRI)) {
-      try {
-        LOG.debug("pid=" + getProcId() + " failed acquire, returning " + LockState.LOCK_EVENT_WAIT +
-            " lock dump " + env.getProcedureScheduler().dumpLocks());
-      } catch (IOException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-      return LockState.LOCK_EVENT_WAIT;
-    }
-    this.lock = true;
-    return LockState.LOCK_ACQUIRED;
-  }
-
-  @Override
-  protected void releaseLock(final MasterProcedureEnv env) {
-    this.lock = false;
-    env.getProcedureScheduler().wakeRegions(this, getTableName(), parentHRI);
-  }
-
-  @Override
-  public TableName getTableName() {
-    return parentHRI.getTable();
+  private HRegionInfo getParentRegion() {
+    return getRegion();
   }
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.SPLIT;
+    return TableOperationType.REGION_SPLIT;
   }
 
   private byte[] getSplitRow() {
@@ -369,12 +336,14 @@ public class SplitTableRegionProcedure
   @VisibleForTesting
   public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException {
     // Check whether the region is splittable
-    RegionStateNode node = env.getAssignmentManager().getRegionStates().getRegionNode(parentHRI);
+    RegionStateNode node = env.getAssignmentManager().getRegionStates().getRegionNode(getParentRegion());
+    HRegionInfo parentHRI = null;
     if (node != null) {
       parentHRI = node.getRegionInfo();
 
       // expected parent to be online or closed
       if (!node.isInState(EXPECTED_SPLIT_STATES)) {
+        // We may have SPLIT already?
         setFailure(new IOException("Split " + parentHRI.getRegionNameAsString() +
             " FAILED because state=" + node.getState() + "; expected " +
             Arrays.toString(EXPECTED_SPLIT_STATES)));
@@ -387,13 +356,32 @@ public class SplitTableRegionProcedure
             "offline/split already."));
         return false;
       }
+
+      // Ask the remote regionserver if this region is splittable. If we get an IOE, report it
+      // along w/ the failure so can see why we are not splittable at this time.
+      IOException splittableCheckIOE = null;
+      boolean splittable = false;
+      try {
+        GetRegionInfoResponse response =
+            Util.getRegionInfoResponse(env, node.getRegionLocation(), node.getRegionInfo());
+        splittable = response.hasSplittable() && response.getSplittable();
+      } catch (IOException e) {
+        splittableCheckIOE = e;
+      }
+      if (!splittable) {
+        IOException e = new IOException(parentHRI.getShortNameToLog() + " NOT splittable");
+        if (splittableCheckIOE != null) e.initCause(splittableCheckIOE);
+        setFailure(e);
+        return false;
+      }
     }
 
-    // since we have the lock and the master is coordinating the operation
+    // Since we have the lock and the master is coordinating the operation
     // we are always able to split the region
     if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
       LOG.warn("pid=" + getProcId() + " split switch is off! skip split of " + parentHRI);
-      setFailure(new IOException("Split region " + parentHRI.getRegionNameAsString() +
+      setFailure(new IOException("Split region " +
+          (parentHRI == null? "null": parentHRI.getRegionNameAsString()) +
           " failed due to split switch off"));
       return false;
     }
@@ -438,7 +426,7 @@ public class SplitTableRegionProcedure
 
     final AssignProcedure[] procs = new AssignProcedure[regionReplication];
     for (int i = 0; i < regionReplication; ++i) {
-      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(parentHRI, i);
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
       procs[i] = env.getAssignmentManager().createAssignProcedure(hri, serverName);
     }
     env.getMasterServices().getMasterProcedureExecutor().submitProcedures(procs);
@@ -452,10 +440,10 @@ public class SplitTableRegionProcedure
   @VisibleForTesting
   public void createDaughterRegions(final MasterProcedureEnv env) throws IOException {
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), parentHRI.getTable());
+    final Path tabledir = FSUtils.getTableDir(mfs.getRootDir(), getTableName());
     final FileSystem fs = mfs.getFileSystem();
     HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(
-      env.getMasterConfiguration(), fs, tabledir, parentHRI, false);
+      env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false);
     regionFs.createSplitsDir();
 
     Pair<Integer, Integer> expectedReferences = splitStoreFiles(env, regionFs);
@@ -509,7 +497,7 @@ public class SplitTableRegionProcedure
         conf.getInt(HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT)),
       nbFiles);
     LOG.info("pid=" + getProcId() + " preparing to split " + nbFiles + " storefiles for region " +
-      parentHRI + " using " + maxThreads + " threads");
+      getParentRegion().getShortNameToLog() + " using " + maxThreads + " threads");
     final ExecutorService threadPool = Executors.newFixedThreadPool(
       maxThreads, Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
     final List<Future<Pair<Path,Path>>> futures = new ArrayList<Future<Pair<Path,Path>>>(nbFiles);
@@ -567,9 +555,8 @@ public class SplitTableRegionProcedure
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("pid=" + getProcId() + " split storefiles for region " + parentHRI + " Daughter A: " +
-          daughterA
-          + " storefiles, Daughter B: " + daughterB + " storefiles.");
+      LOG.debug("pid=" + getProcId() + " split storefiles for region " + getParentRegion().getShortNameToLog() +
+          " Daughter A: " + daughterA + " storefiles, Daughter B: " + daughterB + " storefiles.");
     }
     return new Pair<Integer, Integer>(daughterA, daughterB);
   }
@@ -586,7 +573,7 @@ public class SplitTableRegionProcedure
       final byte[] family, final StoreFile sf) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("pid=" + getProcId() + " splitting started for store file: " +
-          sf.getPath() + " for region: " + parentHRI);
+          sf.getPath() + " for region: " + getParentRegion());
     }
 
     final byte[] splitRow = getSplitRow();
@@ -597,7 +584,7 @@ public class SplitTableRegionProcedure
         regionFs.splitStoreFile(this.daughter_2_HRI, familyName, sf, splitRow, true, null);
     if (LOG.isDebugEnabled()) {
       LOG.debug("pid=" + getProcId() + " splitting complete for store file: " +
-          sf.getPath() + " for region: " + parentHRI);
+          sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
     }
     return new Pair<Path,Path>(path_first, path_second);
   }
@@ -640,7 +627,7 @@ public class SplitTableRegionProcedure
     if (cpHost != null) {
       if (cpHost.preSplitBeforePONRAction(getSplitRow(), metaEntries, getUser())) {
         throw new IOException("Coprocessor bypassing region " +
-            parentHRI.getRegionNameAsString() + " split.");
+            getParentRegion().getRegionNameAsString() + " split.");
       }
       try {
         for (Mutation p : metaEntries) {
@@ -661,7 +648,7 @@ public class SplitTableRegionProcedure
    * @throws IOException
    */
   private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
-    env.getAssignmentManager().markRegionAsSplit(parentHRI, getParentRegionServerName(env),
+    env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
       daughter_1_HRI, daughter_2_HRI);
   }
 
@@ -690,14 +677,14 @@ public class SplitTableRegionProcedure
 
   private ServerName getParentRegionServerName(final MasterProcedureEnv env) {
     return env.getMasterServices().getAssignmentManager()
-      .getRegionStates().getRegionServerOfRegion(parentHRI);
+      .getRegionStates().getRegionServerOfRegion(getParentRegion());
   }
 
   private UnassignProcedure[] createUnassignProcedures(final MasterProcedureEnv env,
       final int regionReplication) {
     final UnassignProcedure[] procs = new UnassignProcedure[regionReplication];
     for (int i = 0; i < procs.length; ++i) {
-      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(parentHRI, i);
+      final HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(getParentRegion(), i);
       procs[i] = env.getAssignmentManager().createUnassignProcedure(hri, null, true);
     }
     return procs;
@@ -735,14 +722,4 @@ public class SplitTableRegionProcedure
     }
     return traceEnabled;
   }
-
-  @Override
-  protected boolean holdLock(final MasterProcedureEnv env) {
-    return true;
-  }
-
-  @Override
-  protected boolean hasLock(final MasterProcedureEnv env) {
-    return lock;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 01570a4..a82a2f5 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
@@ -65,6 +65,11 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 public class UnassignProcedure extends RegionTransitionProcedure {
   private static final Log LOG = LogFactory.getLog(UnassignProcedure.class);
 
+  /**
+   * Where to send the unassign RPC.
+   */
+  protected volatile ServerName destinationServer;
+
   private final AtomicBoolean serverCrashed = new AtomicBoolean(false);
 
   // TODO: should this be in a reassign procedure?
@@ -77,9 +82,9 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   }
 
   public UnassignProcedure(final HRegionInfo regionInfo,
-      final ServerName server, final boolean force) {
+      final ServerName destinationServer, final boolean force) {
     super(regionInfo);
-    this.server = server;
+    this.destinationServer = destinationServer;
     this.force = force;
 
     // we don't need REGION_TRANSITION_QUEUE, we jump directly to sending the request
@@ -88,7 +93,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
 
   @Override
   public TableOperationType getTableOperationType() {
-    return TableOperationType.UNASSIGN;
+    return TableOperationType.REGION_UNASSIGN;
   }
 
   @Override
@@ -106,7 +111,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   public void serializeStateData(final OutputStream stream) throws IOException {
     UnassignRegionStateData.Builder state = UnassignRegionStateData.newBuilder()
         .setTransitionState(getTransitionState())
-        .setDestinationServer(ProtobufUtil.toServerName(server))
+        .setDestinationServer(ProtobufUtil.toServerName(destinationServer))
         .setRegionInfo(HRegionInfo.convert(getRegionInfo()));
     if (force) {
       state.setForce(true);
@@ -121,7 +126,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
     setRegionInfo(HRegionInfo.convert(state.getRegionInfo()));
     force = state.getForce();
     if (state.hasDestinationServer()) {
-      server = ProtobufUtil.toServerName(state.getDestinationServer());
+      this.destinationServer = ProtobufUtil.toServerName(state.getDestinationServer());
     }
   }
 
@@ -177,7 +182,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
   @Override
   public RemoteOperation remoteCallBuild(final MasterProcedureEnv env, final ServerName serverName) {
     assert serverName.equals(getRegionState(env).getRegionLocation());
-    return new RegionCloseOperation(this, getRegionInfo(), server);
+    return new RegionCloseOperation(this, getRegionInfo(), destinationServer);
   }
 
   @Override
@@ -228,4 +233,10 @@ public class UnassignProcedure extends RegionTransitionProcedure {
       serverCrashed.set(true);
     }
   }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    super.toStringClassDetails(sb);
+    sb.append(", server=").append(this.destinationServer);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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
new file mode 100644
index 0000000..cb3861a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/Util.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.assignment;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+
+/**
+ * Utility for this assignment package only.
+ */
+@InterfaceAudience.Private
+class Util {
+  private Util() {}
+
+  /**
+   * Raw call to remote regionserver to get info on a particular region.
+   * @throws IOException Let it out so can report this IOE as reason for failure
+   */
+  static GetRegionInfoResponse getRegionInfoResponse(final MasterProcedureEnv env,
+      final ServerName regionLocation, final HRegionInfo hri)
+  throws IOException {
+    // TODO: There is no timeout on this controller. Set one!
+    HBaseRpcController controller = env.getMasterServices().getClusterConnection().
+        getRpcControllerFactory().newController();
+    final AdminService.BlockingInterface admin =
+        env.getMasterServices().getClusterConnection().getAdmin(regionLocation);
+    GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(hri.getRegionName());
+    try {
+      return admin.getRegionInfo(controller, request);
+    } catch (ServiceException e) {
+      throw ProtobufUtil.handleRemoteException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index 6a6c899..a494ecc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -599,8 +599,6 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     /**
      * Return true if the placement of region on server would lower the availability
      * of the region in question
-     * @param server
-     * @param region
      * @return true or false
      */
     boolean wouldLowerAvailability(HRegionInfo regionInfo, ServerName serverName) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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
new file mode 100644
index 0000000..159f210
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineRegionProcedure.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Base class for all the Region procedures that want to use a StateMachine.
+ * It provides some basic helpers like basic locking, sync latch, and toStringClassDetails().
+ * Defaults to holding the lock for the life of the procedure.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractStateMachineRegionProcedure<TState>
+    extends AbstractStateMachineTableProcedure<TState> {
+  private HRegionInfo hri;
+  private volatile boolean lock = false;
+
+  public AbstractStateMachineRegionProcedure(final MasterProcedureEnv env,
+      final HRegionInfo hri) {
+    super(env);
+    this.hri = hri;
+  }
+
+  public AbstractStateMachineRegionProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    super();
+  }
+
+  /**
+   * @return The HRegionInfo of the region we are operating on.
+   */
+  protected HRegionInfo getRegion() {
+    return this.hri;
+  }
+
+  /**
+   * Used when deserializing. Otherwise, DON'T TOUCH IT!
+   */
+  protected void setRegion(final HRegionInfo hri) {
+    this.hri = hri;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return getRegion().getTable();
+  }
+
+  @Override
+  public abstract TableOperationType getTableOperationType();
+
+  @Override
+  public void toStringClassDetails(final StringBuilder sb) {
+    super.toStringClassDetails(sb);
+    sb.append(", region=").append(getRegion().getShortNameToLog());
+  }
+
+  /**
+   * Check whether a table is modifiable - exists and either offline or online with config set
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
+    // Checks whether the table exists
+    if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
+      throw new TableNotFoundException(getTableName());
+    }
+  }
+
+  @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  protected LockState acquireLock(final MasterProcedureEnv env) {
+    if (env.waitInitialized(this)) return LockState.LOCK_EVENT_WAIT;
+    if (env.getProcedureScheduler().waitRegions(this, getTableName(), getRegion())) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    this.lock = true;
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  protected void releaseLock(final MasterProcedureEnv env) {
+    this.lock = false;
+    env.getProcedureScheduler().wakeRegions(this, getTableName(), getRegion());
+  }
+
+  @Override
+  protected boolean hasLock(final MasterProcedureEnv env) {
+    return this.lock;
+  }
+
+  protected void setFailure(Throwable cause) {
+    super.setFailure(getClass().getSimpleName(), cause);
+  }
+}
\ No newline at end of file


[3/4] hbase git commit: Fix CatalogTracker. Make it use Procedures doing clean up of Region data on split/merge. Without these changes, ITBLL was failing at larger scale (3-4hours 5B rows) because we were splitting split Regions.

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
index a5e2eaa..88677e4 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/MasterProcedureProtos.java
@@ -2515,6 +2515,204 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(enum_scope:hbase.pb.MoveRegionState)
   }
 
+  /**
+   * Protobuf enum {@code hbase.pb.GCRegionState}
+   */
+  public enum GCRegionState
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>GC_REGION_PREPARE = 1;</code>
+     */
+    GC_REGION_PREPARE(1),
+    /**
+     * <code>GC_REGION_ARCHIVE = 2;</code>
+     */
+    GC_REGION_ARCHIVE(2),
+    /**
+     * <code>GC_REGION_PURGE_METADATA = 3;</code>
+     */
+    GC_REGION_PURGE_METADATA(3),
+    ;
+
+    /**
+     * <code>GC_REGION_PREPARE = 1;</code>
+     */
+    public static final int GC_REGION_PREPARE_VALUE = 1;
+    /**
+     * <code>GC_REGION_ARCHIVE = 2;</code>
+     */
+    public static final int GC_REGION_ARCHIVE_VALUE = 2;
+    /**
+     * <code>GC_REGION_PURGE_METADATA = 3;</code>
+     */
+    public static final int GC_REGION_PURGE_METADATA_VALUE = 3;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static GCRegionState valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static GCRegionState forNumber(int value) {
+      switch (value) {
+        case 1: return GC_REGION_PREPARE;
+        case 2: return GC_REGION_ARCHIVE;
+        case 3: return GC_REGION_PURGE_METADATA;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCRegionState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        GCRegionState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCRegionState>() {
+            public GCRegionState findValueByNumber(int number) {
+              return GCRegionState.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(20);
+    }
+
+    private static final GCRegionState[] VALUES = values();
+
+    public static GCRegionState valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private GCRegionState(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.GCRegionState)
+  }
+
+  /**
+   * Protobuf enum {@code hbase.pb.GCMergedRegionsState}
+   */
+  public enum GCMergedRegionsState
+      implements org.apache.hadoop.hbase.shaded.com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>GC_MERGED_REGIONS_PREPARE = 1;</code>
+     */
+    GC_MERGED_REGIONS_PREPARE(1),
+    /**
+     * <code>GC_MERGED_REGIONS_PURGE = 2;</code>
+     */
+    GC_MERGED_REGIONS_PURGE(2),
+    /**
+     * <code>GC_REGION_EDIT_METADATA = 3;</code>
+     */
+    GC_REGION_EDIT_METADATA(3),
+    ;
+
+    /**
+     * <code>GC_MERGED_REGIONS_PREPARE = 1;</code>
+     */
+    public static final int GC_MERGED_REGIONS_PREPARE_VALUE = 1;
+    /**
+     * <code>GC_MERGED_REGIONS_PURGE = 2;</code>
+     */
+    public static final int GC_MERGED_REGIONS_PURGE_VALUE = 2;
+    /**
+     * <code>GC_REGION_EDIT_METADATA = 3;</code>
+     */
+    public static final int GC_REGION_EDIT_METADATA_VALUE = 3;
+
+
+    public final int getNumber() {
+      return value;
+    }
+
+    /**
+     * @deprecated Use {@link #forNumber(int)} instead.
+     */
+    @java.lang.Deprecated
+    public static GCMergedRegionsState valueOf(int value) {
+      return forNumber(value);
+    }
+
+    public static GCMergedRegionsState forNumber(int value) {
+      switch (value) {
+        case 1: return GC_MERGED_REGIONS_PREPARE;
+        case 2: return GC_MERGED_REGIONS_PURGE;
+        case 3: return GC_REGION_EDIT_METADATA;
+        default: return null;
+      }
+    }
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCMergedRegionsState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<
+        GCMergedRegionsState> internalValueMap =
+          new org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLiteMap<GCMergedRegionsState>() {
+            public GCMergedRegionsState findValueByNumber(int number) {
+              return GCMergedRegionsState.forNumber(number);
+            }
+          };
+
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(ordinal());
+    }
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(21);
+    }
+
+    private static final GCMergedRegionsState[] VALUES = values();
+
+    public static GCMergedRegionsState valueOf(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int value;
+
+    private GCMergedRegionsState(int value) {
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:hbase.pb.GCMergedRegionsState)
+  }
+
   public interface CreateTableStateDataOrBuilder extends
       // @@protoc_insertion_point(interface_extends:hbase.pb.CreateTableStateData)
       org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
@@ -28538,122 +28736,1742 @@ public final class MasterProcedureProtos {
 
   }
 
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_CreateTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_CreateTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ModifyTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ModifyTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_TruncateTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_TruncateTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_DeleteTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_DeleteTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_CreateNamespaceStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_CreateNamespaceStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ModifyNamespaceStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ModifyNamespaceStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_DeleteNamespaceStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_DeleteNamespaceStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_AddColumnFamilyStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_AddColumnFamilyStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ModifyColumnFamilyStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ModifyColumnFamilyStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_DeleteColumnFamilyStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_DeleteColumnFamilyStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_EnableTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_EnableTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_DisableTableStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_DisableTableStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_RestoreParentToChildRegionsPair_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_RestoreParentToChildRegionsPair_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_CloneSnapshotStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_CloneSnapshotStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_RestoreSnapshotStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_RestoreSnapshotStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_DispatchMergingRegionsStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_DispatchMergingRegionsStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_SplitTableRegionStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_SplitTableRegionStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_MergeTableRegionsStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_MergeTableRegionsStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_ServerCrashStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_ServerCrashStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_AssignRegionStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_AssignRegionStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_UnassignRegionStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_UnassignRegionStateData_fieldAccessorTable;
-  private static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-    internal_static_hbase_pb_MoveRegionStateData_descriptor;
-  private static final 
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-      internal_static_hbase_pb_MoveRegionStateData_fieldAccessorTable;
+  public interface GCRegionStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GCRegionStateData)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
 
-  public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static  org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    boolean hasRegionInfo();
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo();
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GCRegionStateData}
+   */
+  public  static final class GCRegionStateData extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GCRegionStateData)
+      GCRegionStateDataOrBuilder {
+    // Use GCRegionStateData.newBuilder() to construct.
+    private GCRegionStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GCRegionStateData() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GCRegionStateData(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = regionInfo_.toBuilder();
+              }
+              regionInfo_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(regionInfo_);
+                regionInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCRegionStateData_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCRegionStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int REGION_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_;
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    public boolean hasRegionInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
+      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
+      return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasRegionInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getRegionInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getRegionInfo());
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getRegionInfo());
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData) obj;
+
+      boolean result = true;
+      result = result && (hasRegionInfo() == other.hasRegionInfo());
+      if (hasRegionInfo()) {
+        result = result && getRegionInfo()
+            .equals(other.getRegionInfo());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasRegionInfo()) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfo().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GCRegionStateData}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.GCRegionStateData)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateDataOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCRegionStateData_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCRegionStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getRegionInfoFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = null;
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCRegionStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (regionInfoBuilder_ == null) {
+          result.regionInfo_ = regionInfo_;
+        } else {
+          result.regionInfo_ = regionInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData.getDefaultInstance()) return this;
+        if (other.hasRegionInfo()) {
+          mergeRegionInfo(other.getRegionInfo());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasRegionInfo()) {
+          return false;
+        }
+        if (!getRegionInfo().isInitialized()) {
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo regionInfo_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public boolean hasRegionInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
+        } else {
+          return regionInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public Builder setRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          regionInfo_ = value;
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public Builder setRegionInfo(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public Builder mergeRegionInfo(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              regionInfo_ != null &&
+              regionInfo_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            regionInfo_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(regionInfo_).mergeFrom(value).buildPartial();
+          } else {
+            regionInfo_ = value;
+          }
+          onChanged();
+        } else {
+          regionInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = null;
+          onChanged();
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getRegionInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return regionInfo_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : regionInfo_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo region_info = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  getRegionInfo(),
+                  getParentForChildren(),
+                  isClean());
+          regionInfo_ = null;
+        }
+        return regionInfoBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GCRegionStateData)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GCRegionStateData)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GCRegionStateData>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<GCRegionStateData>() {
+      public GCRegionStateData parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new GCRegionStateData(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GCRegionStateData> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GCRegionStateData> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionStateData getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface GCMergedRegionsStateDataOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.GCMergedRegionsStateData)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    boolean hasParentA();
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentA();
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentAOrBuilder();
+
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    boolean hasParentB();
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentB();
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentBOrBuilder();
+
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    boolean hasMergedChild();
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedChild();
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedChildOrBuilder();
+  }
+  /**
+   * Protobuf type {@code hbase.pb.GCMergedRegionsStateData}
+   */
+  public  static final class GCMergedRegionsStateData extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.GCMergedRegionsStateData)
+      GCMergedRegionsStateDataOrBuilder {
+    // Use GCMergedRegionsStateData.newBuilder() to construct.
+    private GCMergedRegionsStateData(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private GCMergedRegionsStateData() {
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GCMergedRegionsStateData(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = parentA_.toBuilder();
+              }
+              parentA_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(parentA_);
+                parentA_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = parentB_.toBuilder();
+              }
+              parentB_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(parentB_);
+                parentB_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = mergedChild_.toBuilder();
+              }
+              mergedChild_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(mergedChild_);
+                mergedChild_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCMergedRegionsStateData_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCMergedRegionsStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int PARENT_A_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentA_;
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    public boolean hasParentA() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentA() {
+      return parentA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentA_;
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentAOrBuilder() {
+      return parentA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentA_;
+    }
+
+    public static final int PARENT_B_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentB_;
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    public boolean hasParentB() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentB() {
+      return parentB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentB_;
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentBOrBuilder() {
+      return parentB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentB_;
+    }
+
+    public static final int MERGED_CHILD_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo mergedChild_;
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    public boolean hasMergedChild() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedChild() {
+      return mergedChild_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedChild_;
+    }
+    /**
+     * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedChildOrBuilder() {
+      return mergedChild_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedChild_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasParentA()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasParentB()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasMergedChild()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getParentA().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getParentB().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getMergedChild().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getParentA());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, getParentB());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, getMergedChild());
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getParentA());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, getParentB());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, getMergedChild());
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData other = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData) obj;
+
+      boolean result = true;
+      result = result && (hasParentA() == other.hasParentA());
+      if (hasParentA()) {
+        result = result && getParentA()
+            .equals(other.getParentA());
+      }
+      result = result && (hasParentB() == other.hasParentB());
+      if (hasParentB()) {
+        result = result && getParentB()
+            .equals(other.getParentB());
+      }
+      result = result && (hasMergedChild() == other.hasMergedChild());
+      if (hasMergedChild()) {
+        result = result && getMergedChild()
+            .equals(other.getMergedChild());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptor().hashCode();
+      if (hasParentA()) {
+        hash = (37 * hash) + PARENT_A_FIELD_NUMBER;
+        hash = (53 * hash) + getParentA().hashCode();
+      }
+      if (hasParentB()) {
+        hash = (37 * hash) + PARENT_B_FIELD_NUMBER;
+        hash = (53 * hash) + getParentB().hashCode();
+      }
+      if (hasMergedChild()) {
+        hash = (37 * hash) + MERGED_CHILD_FIELD_NUMBER;
+        hash = (53 * hash) + getMergedChild().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.GCMergedRegionsStateData}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.GCMergedRegionsStateData)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateDataOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCMergedRegionsStateData_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCMergedRegionsStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.class, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getParentAFieldBuilder();
+          getParentBFieldBuilder();
+          getMergedChildFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (parentABuilder_ == null) {
+          parentA_ = null;
+        } else {
+          parentABuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (parentBBuilder_ == null) {
+          parentB_ = null;
+        } else {
+          parentBBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (mergedChildBuilder_ == null) {
+          mergedChild_ = null;
+        } else {
+          mergedChildBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.internal_static_hbase_pb_GCMergedRegionsStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData result = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (parentABuilder_ == null) {
+          result.parentA_ = parentA_;
+        } else {
+          result.parentA_ = parentABuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (parentBBuilder_ == null) {
+          result.parentB_ = parentB_;
+        } else {
+          result.parentB_ = parentBBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (mergedChildBuilder_ == null) {
+          result.mergedChild_ = mergedChild_;
+        } else {
+          result.mergedChild_ = mergedChildBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData.getDefaultInstance()) return this;
+        if (other.hasParentA()) {
+          mergeParentA(other.getParentA());
+        }
+        if (other.hasParentB()) {
+          mergeParentB(other.getParentB());
+        }
+        if (other.hasMergedChild()) {
+          mergeMergedChild(other.getMergedChild());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasParentA()) {
+          return false;
+        }
+        if (!hasParentB()) {
+          return false;
+        }
+        if (!hasMergedChild()) {
+          return false;
+        }
+        if (!getParentA().isInitialized()) {
+          return false;
+        }
+        if (!getParentB().isInitialized()) {
+          return false;
+        }
+        if (!getMergedChild().isInitialized()) {
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentA_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> parentABuilder_;
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public boolean hasParentA() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentA() {
+        if (parentABuilder_ == null) {
+          return parentA_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentA_;
+        } else {
+          return parentABuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public Builder setParentA(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (parentABuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          parentA_ = value;
+          onChanged();
+        } else {
+          parentABuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public Builder setParentA(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (parentABuilder_ == null) {
+          parentA_ = builderForValue.build();
+          onChanged();
+        } else {
+          parentABuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public Builder mergeParentA(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (parentABuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              parentA_ != null &&
+              parentA_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            parentA_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(parentA_).mergeFrom(value).buildPartial();
+          } else {
+            parentA_ = value;
+          }
+          onChanged();
+        } else {
+          parentABuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public Builder clearParentA() {
+        if (parentABuilder_ == null) {
+          parentA_ = null;
+          onChanged();
+        } else {
+          parentABuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getParentABuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getParentAFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentAOrBuilder() {
+        if (parentABuilder_ != null) {
+          return parentABuilder_.getMessageOrBuilder();
+        } else {
+          return parentA_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentA_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_a = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getParentAFieldBuilder() {
+        if (parentABuilder_ == null) {
+          parentABuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  getParentA(),
+                  getParentForChildren(),
+                  isClean());
+          parentA_ = null;
+        }
+        return parentABuilder_;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo parentB_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> parentBBuilder_;
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public boolean hasParentB() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getParentB() {
+        if (parentBBuilder_ == null) {
+          return parentB_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentB_;
+        } else {
+          return parentBBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public Builder setParentB(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (parentBBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          parentB_ = value;
+          onChanged();
+        } else {
+          parentBBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public Builder setParentB(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (parentBBuilder_ == null) {
+          parentB_ = builderForValue.build();
+          onChanged();
+        } else {
+          parentBBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public Builder mergeParentB(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (parentBBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              parentB_ != null &&
+              parentB_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            parentB_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(parentB_).mergeFrom(value).buildPartial();
+          } else {
+            parentB_ = value;
+          }
+          onChanged();
+        } else {
+          parentBBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public Builder clearParentB() {
+        if (parentBBuilder_ == null) {
+          parentB_ = null;
+          onChanged();
+        } else {
+          parentBBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getParentBBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getParentBFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getParentBOrBuilder() {
+        if (parentBBuilder_ != null) {
+          return parentBBuilder_.getMessageOrBuilder();
+        } else {
+          return parentB_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : parentB_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo parent_b = 2;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getParentBFieldBuilder() {
+        if (parentBBuilder_ == null) {
+          parentBBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  getParentB(),
+                  getParentForChildren(),
+                  isClean());
+          parentB_ = null;
+        }
+        return parentBBuilder_;
+      }
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo mergedChild_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> mergedChildBuilder_;
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public boolean hasMergedChild() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo getMergedChild() {
+        if (mergedChildBuilder_ == null) {
+          return mergedChild_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedChild_;
+        } else {
+          return mergedChildBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public Builder setMergedChild(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (mergedChildBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          mergedChild_ = value;
+          onChanged();
+        } else {
+          mergedChildBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public Builder setMergedChild(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (mergedChildBuilder_ == null) {
+          mergedChild_ = builderForValue.build();
+          onChanged();
+        } else {
+          mergedChildBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public Builder mergeMergedChild(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (mergedChildBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              mergedChild_ != null &&
+              mergedChild_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance()) {
+            mergedChild_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.newBuilder(mergedChild_).mergeFrom(value).buildPartial();
+          } else {
+            mergedChild_ = value;
+          }
+          onChanged();
+        } else {
+          mergedChildBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public Builder clearMergedChild() {
+        if (mergedChildBuilder_ == null) {
+          mergedChild_ = null;
+          onChanged();
+        } else {
+          mergedChildBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder getMergedChildBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getMergedChildFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getMergedChildOrBuilder() {
+        if (mergedChildBuilder_ != null) {
+          return mergedChildBuilder_.getMessageOrBuilder();
+        } else {
+          return mergedChild_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance() : mergedChild_;
+        }
+      }
+      /**
+       * <code>required .hbase.pb.RegionInfo merged_child = 3;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getMergedChildFieldBuilder() {
+        if (mergedChildBuilder_ == null) {
+          mergedChildBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  getMergedChild(),
+                  getParentForChildren(),
+                  isClean());
+          mergedChild_ = null;
+        }
+        return mergedChildBuilder_;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.GCMergedRegionsStateData)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.GCMergedRegionsStateData)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCMergedRegionsStateData getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<GCMergedRegionsStateData>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<GCMergedRegionsStateData>() {
+      public GCMergedRegionsStateData parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new GCMergedRegionsStateData(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.

<TRUNCATED>

[4/4] hbase git commit: Fix CatalogTracker. Make it use Procedures doing clean up of Region data on split/merge. Without these changes, ITBLL was failing at larger scale (3-4hours 5B rows) because we were splitting split Regions.

Posted by st...@apache.org.
Fix CatalogTracker. Make it use Procedures doing clean up of Region
data on split/merge. Without these changes, ITBLL was failing at
larger scale (3-4hours 5B rows) because we were splitting split
Regions.

Added a bunch of doc. on Procedure primitives.

Added new region-based state machine base class. Moved region-based
state machines on to it.

Found bugs in the way procedure locking was doing in a few of the
region-based Procedures. Having them all have same subclass helps here.

Added isSplittable and isMergeable to the Region Interface.

Master would split/merge even though the Regions still had
references. Fixed it so Master asks RegionServer if Region
is splittable.

Messing more w/ logging. Made all procedures log the same and report
the state the same; helps when logging is regular.

Rewrote TestCatalogTracker. Enabled TestMergeTableRegionProcedure.

Added more functionality to MockMasterServices so can use it doing
standalone testing of Procedures (made TestCatalogTracker use it
instead of its own version).


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

Branch: refs/heads/HBASE-14614
Commit: a23fcc97dea57d58bfc3fc0ea97d8e9adaf8be8a
Parents: 9464f46
Author: Michael Stack <st...@apache.org>
Authored: Thu May 11 16:59:27 2017 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu May 11 17:14:02 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HRegionInfo.java    |    4 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  |    7 +-
 .../hadoop/hbase/procedure2/Procedure.java      |  161 +-
 .../hbase/procedure2/ProcedureExecutor.java     |   36 +-
 .../hbase/procedure2/StateMachineProcedure.java |    6 +-
 .../shaded/protobuf/generated/AdminProtos.java  |  483 +++-
 .../generated/MasterProcedureProtos.java        | 2318 ++++++++++++++++--
 .../shaded/protobuf/generated/MasterProtos.java |   32 +
 .../src/main/protobuf/Admin.proto               |    4 +
 .../src/main/protobuf/Master.proto              |    1 +
 .../src/main/protobuf/MasterProcedure.proto     |   22 +
 .../hadoop/hbase/backup/HFileArchiver.java      |   15 +-
 .../hadoop/hbase/master/CatalogJanitor.java     |   79 +-
 .../hadoop/hbase/master/TableStateManager.java  |    3 +-
 .../master/assignment/AssignProcedure.java      |   61 +-
 .../assignment/GCMergedRegionsProcedure.java    |  170 ++
 .../master/assignment/GCRegionProcedure.java    |  154 ++
 .../assignment/MergeTableRegionsProcedure.java  |  131 +-
 .../master/assignment/MoveRegionProcedure.java  |   22 +-
 .../master/assignment/RegionStateStore.java     |    8 +-
 .../hbase/master/assignment/RegionStates.java   |   12 +-
 .../assignment/RegionTransitionProcedure.java   |   21 +-
 .../assignment/SplitTableRegionProcedure.java   |  125 +-
 .../master/assignment/UnassignProcedure.java    |   23 +-
 .../hadoop/hbase/master/assignment/Util.java    |   60 +
 .../hbase/master/balancer/BaseLoadBalancer.java |    2 -
 .../AbstractStateMachineRegionProcedure.java    |  118 +
 .../AbstractStateMachineTableProcedure.java     |   11 +-
 .../DispatchMergingRegionsProcedure.java        |    2 +-
 .../procedure/MasterProcedureScheduler.java     |   10 +-
 .../master/procedure/ServerCrashProcedure.java  |    8 +-
 .../procedure/TableProcedureInterface.java      |    3 +-
 .../hadoop/hbase/regionserver/HRegion.java      |    6 +-
 .../hbase/regionserver/HRegionFileSystem.java   |    3 +-
 .../hbase/regionserver/RSRpcServices.java       |    2 +
 .../hadoop/hbase/regionserver/Region.java       |    8 +
 .../hadoop/hbase/HBaseTestingUtility.java       |    2 +-
 .../hbase/client/TestAsyncRegionAdminApi.java   |    3 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |  596 ++---
 .../master/assignment/MockMasterServices.java   |  184 +-
 .../TestMergeTableRegionsProcedure.java         |   44 +-
 .../TestSplitTableRegionProcedure.java          |   20 +-
 ...ProcedureSchedulerPerformanceEvaluation.java |    2 +-
 .../procedure/TestMasterProcedureScheduler.java |   20 +-
 44 files changed, 3839 insertions(+), 1163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 5b9cbec..d470ffa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -168,6 +168,10 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
     return prettyPrint(this.getEncodedName());
   }
 
+  public static String getShortNameToLog(HRegionInfo...hris) {
+    return getShortNameToLog(Arrays.asList(hris));
+  }
+
   /**
    * @return Return a String of short, printable names for <code>hris</code>
    * (usually encoded name) for us logging.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 9f1be9f..9eb5111 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1663,8 +1663,11 @@ public class MetaTableAccessor {
       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
       Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
 
-      // The merged is a new region, openSeqNum = 1 is fine.
-      addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
+      // The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
+      // if crash after merge happened but before we got to here.. means in-memory
+      // locations of offlined merged, now-closed, regions is lost. Should be ok. We
+      // assign the merged region later.
+      if (sn != null) addLocation(putOfMerged, sn, 1, -1, mergedRegion.getReplicaId());
 
       // Add empty locations for region replicas of the merged region so that number of replicas can
       // be cached whenever the primary region is looked up from meta

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 ffb09c9..f065a98 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
@@ -1104,6 +1104,7 @@ public class ProcedureExecutor<TEnvironment> {
               scheduler.yield(proc);
               break;
             case LOCK_EVENT_WAIT:
+              LOG.info("DEBUG LOCK_EVENT_WAIT rollback..." + proc);
               procStack.unsetRollback();
               break;
             default:
@@ -1121,6 +1122,7 @@ public class ProcedureExecutor<TEnvironment> {
                 scheduler.yield(proc);
                 break;
               case LOCK_EVENT_WAIT:
+                LOG.info("DEBUG LOCK_EVENT_WAIT can't rollback child running?..." + proc);
                 break;
               default:
                 throw new UnsupportedOperationException();
@@ -1372,7 +1374,9 @@ public class ProcedureExecutor<TEnvironment> {
           subprocs = null;
         }
       } catch (ProcedureSuspendedException e) {
-        LOG.info("Suspend " + procedure);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Suspend " + procedure);
+        }
         suspended = true;
       } catch (ProcedureYieldException e) {
         if (LOG.isTraceEnabled()) {
@@ -1397,11 +1401,13 @@ public class ProcedureExecutor<TEnvironment> {
       if (!procedure.isFailed()) {
         if (subprocs != null) {
           if (subprocs.length == 1 && subprocs[0] == procedure) {
-            // Procedure returned itself.
-            // Quick-shortcut for a state machine like procedure
+            // Procedure returned itself. Quick-shortcut for a state machine-like procedure;
+            // i.e. we go around this loop again rather than go back out on the scheduler queue.
             subprocs = null;
             reExecute = true;
-            LOG.info("Short-circuit to rexecute for pid=" + procedure.getProcId());
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Short-circuit to next step on pid=" + procedure.getProcId());
+            }
           } else {
             // Yield the current procedure, and make the subprocedure runnable
             // subprocs may come back 'null'.
@@ -1529,7 +1535,7 @@ public class ProcedureExecutor<TEnvironment> {
       store.update(parent);
       scheduler.addFront(parent);
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Finished ALL subprocedures of " + parent + "; resume.");
+        LOG.debug("Finished subprocedure(s) of " + parent + "; resume parent processing.");
       }
       return;
     }
@@ -1619,6 +1625,7 @@ public class ProcedureExecutor<TEnvironment> {
   // ==========================================================================
   private final class WorkerThread extends StoppableThread {
     private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE);
+    private Procedure activeProcedure;
 
     public WorkerThread(final ThreadGroup group) {
       super(group, "ProcExecWrkr-" + workerId.incrementAndGet());
@@ -1635,27 +1642,28 @@ public class ProcedureExecutor<TEnvironment> {
       long lastUpdate = EnvironmentEdgeManager.currentTime();
       try {
         while (isRunning() && keepAlive(lastUpdate)) {
-          final Procedure procedure = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
-          if (procedure == null) continue;
+          this.activeProcedure = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS);
+          if (this.activeProcedure == null) continue;
           int activeCount = activeExecutorCount.incrementAndGet();
           int runningCount = store.setRunningProcedureCount(activeCount);
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Execute pid=" + procedure.getProcId() +
+            LOG.debug("Execute pid=" + this.activeProcedure.getProcId() +
                 " runningCount=" + runningCount + ", activeCount=" + activeCount);
           }
           executionStartTime.set(EnvironmentEdgeManager.currentTime());
           try {
-            executeProcedure(procedure);
+            executeProcedure(this.activeProcedure);
           } catch (AssertionError e) {
-            LOG.info("ASSERT pid=" + procedure.getProcId(), e);
+            LOG.info("ASSERT pid=" + this.activeProcedure.getProcId(), e);
             throw e;
           } finally {
             activeCount = activeExecutorCount.decrementAndGet();
             runningCount = store.setRunningProcedureCount(activeCount);
             if (LOG.isDebugEnabled()) {
-              LOG.debug("Leave pid=" + procedure.getProcId() +
+              LOG.debug("Halt pid=" + this.activeProcedure.getProcId() +
                   " runningCount=" + runningCount + ", activeCount=" + activeCount);
             }
+            this.activeProcedure = null;
             lastUpdate = EnvironmentEdgeManager.currentTime();
             executionStartTime.set(Long.MAX_VALUE);
           }
@@ -1668,6 +1676,12 @@ public class ProcedureExecutor<TEnvironment> {
       workerThreads.remove(this);
     }
 
+    @Override
+    public String toString() {
+      Procedure p = this.activeProcedure;
+      return getName() + "(pid=" + (p == null? Procedure.NO_PROC_ID: p.getProcId() + ")");
+    }
+
     /**
      * @return the time since the current procedure is running
      */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/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 0008c16..900b472 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
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMa
 /**
  * Procedure described by a series of steps.
  *
- * The procedure implementor must have an enum of 'states', describing
+ * <p>The procedure implementor must have an enum of 'states', describing
  * the various step of the procedure.
  * Once the procedure is running, the procedure-framework will call executeFromState()
  * using the 'state' provided by the user. The first call to executeFromState()
@@ -71,7 +71,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
    *         Flow.HAS_MORE_STATE if there is another step.
    */
   protected abstract Flow executeFromState(TEnvironment env, TState state)
-    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException;
+  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException;
 
   /**
    * called to perform the rollback of the specified state
@@ -148,7 +148,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
 
   @Override
   protected Procedure[] execute(final TEnvironment env)
-      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+  throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     updateTimestamp();
     try {
       if (!hasMoreState() || isFailed()) return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a23fcc97/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
index d802523..6205038 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/AdminProtos.java
@@ -728,6 +728,40 @@ public final class AdminProtos {
      * <code>optional bool isRecovering = 3;</code>
      */
     boolean getIsRecovering();
+
+    /**
+     * <pre>
+     * True if region is splittable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool splittable = 4;</code>
+     */
+    boolean hasSplittable();
+    /**
+     * <pre>
+     * True if region is splittable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool splittable = 4;</code>
+     */
+    boolean getSplittable();
+
+    /**
+     * <pre>
+     * True if region is mergeable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool mergeable = 5;</code>
+     */
+    boolean hasMergeable();
+    /**
+     * <pre>
+     * True if region is mergeable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool mergeable = 5;</code>
+     */
+    boolean getMergeable();
   }
   /**
    * Protobuf type {@code hbase.pb.GetRegionInfoResponse}
@@ -743,6 +777,8 @@ public final class AdminProtos {
     private GetRegionInfoResponse() {
       compactionState_ = 0;
       isRecovering_ = false;
+      splittable_ = false;
+      mergeable_ = false;
     }
 
     @java.lang.Override
@@ -802,6 +838,16 @@ public final class AdminProtos {
               isRecovering_ = input.readBool();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              splittable_ = input.readBool();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              mergeable_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
@@ -987,6 +1033,52 @@ public final class AdminProtos {
       return isRecovering_;
     }
 
+    public static final int SPLITTABLE_FIELD_NUMBER = 4;
+    private boolean splittable_;
+    /**
+     * <pre>
+     * True if region is splittable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool splittable = 4;</code>
+     */
+    public boolean hasSplittable() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <pre>
+     * True if region is splittable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool splittable = 4;</code>
+     */
+    public boolean getSplittable() {
+      return splittable_;
+    }
+
+    public static final int MERGEABLE_FIELD_NUMBER = 5;
+    private boolean mergeable_;
+    /**
+     * <pre>
+     * True if region is mergeable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool mergeable = 5;</code>
+     */
+    public boolean hasMergeable() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <pre>
+     * True if region is mergeable, false otherwise.
+     * </pre>
+     *
+     * <code>optional bool mergeable = 5;</code>
+     */
+    public boolean getMergeable() {
+      return mergeable_;
+    }
+
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
@@ -1016,6 +1108,12 @@ public final class AdminProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeBool(3, isRecovering_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(4, splittable_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBool(5, mergeable_);
+      }
       unknownFields.writeTo(output);
     }
 
@@ -1036,6 +1134,14 @@ public final class AdminProtos {
         size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, isRecovering_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, splittable_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeBoolSize(5, mergeable_);
+      }
       size += unknownFields.getSerializedSize();
       memoizedSize = size;
       return size;
@@ -1067,6 +1173,16 @@ public final class AdminProtos {
         result = result && (getIsRecovering()
             == other.getIsRecovering());
       }
+      result = result && (hasSplittable() == other.hasSplittable());
+      if (hasSplittable()) {
+        result = result && (getSplittable()
+            == other.getSplittable());
+      }
+      result = result && (hasMergeable() == other.hasMergeable());
+      if (hasMergeable()) {
+        result = result && (getMergeable()
+            == other.getMergeable());
+      }
       result = result && unknownFields.equals(other.unknownFields);
       return result;
     }
@@ -1091,6 +1207,16 @@ public final class AdminProtos {
         hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
             getIsRecovering());
       }
+      if (hasSplittable()) {
+        hash = (37 * hash) + SPLITTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getSplittable());
+      }
+      if (hasMergeable()) {
+        hash = (37 * hash) + MERGEABLE_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashBoolean(
+            getMergeable());
+      }
       hash = (29 * hash) + unknownFields.hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1220,6 +1346,10 @@ public final class AdminProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         isRecovering_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
+        splittable_ = false;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        mergeable_ = false;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -1260,6 +1390,14 @@ public final class AdminProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.isRecovering_ = isRecovering_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.splittable_ = splittable_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.mergeable_ = mergeable_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1311,6 +1449,12 @@ public final class AdminProtos {
         if (other.hasIsRecovering()) {
           setIsRecovering(other.getIsRecovering());
         }
+        if (other.hasSplittable()) {
+          setSplittable(other.getSplittable());
+        }
+        if (other.hasMergeable()) {
+          setMergeable(other.getMergeable());
+        }
         this.mergeUnknownFields(other.unknownFields);
         onChanged();
         return this;
@@ -1530,6 +1674,102 @@ public final class AdminProtos {
         onChanged();
         return this;
       }
+
+      private boolean splittable_ ;
+      /**
+       * <pre>
+       * True if region is splittable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool splittable = 4;</code>
+       */
+      public boolean hasSplittable() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <pre>
+       * True if region is splittable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool splittable = 4;</code>
+       */
+      public boolean getSplittable() {
+        return splittable_;
+      }
+      /**
+       * <pre>
+       * True if region is splittable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool splittable = 4;</code>
+       */
+      public Builder setSplittable(boolean value) {
+        bitField0_ |= 0x00000008;
+        splittable_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * True if region is splittable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool splittable = 4;</code>
+       */
+      public Builder clearSplittable() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        splittable_ = false;
+        onChanged();
+        return this;
+      }
+
+      private boolean mergeable_ ;
+      /**
+       * <pre>
+       * True if region is mergeable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool mergeable = 5;</code>
+       */
+      public boolean hasMergeable() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <pre>
+       * True if region is mergeable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool mergeable = 5;</code>
+       */
+      public boolean getMergeable() {
+        return mergeable_;
+      }
+      /**
+       * <pre>
+       * True if region is mergeable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool mergeable = 5;</code>
+       */
+      public Builder setMergeable(boolean value) {
+        bitField0_ |= 0x00000010;
+        mergeable_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * True if region is mergeable, false otherwise.
+       * </pre>
+       *
+       * <code>optional bool mergeable = 5;</code>
+       */
+      public Builder clearMergeable() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        mergeable_ = false;
+        onChanged();
+        return this;
+      }
       public final Builder setUnknownFields(
           final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
         return super.setUnknownFields(unknownFields);
@@ -28886,129 +29126,130 @@ public final class AdminProtos {
       "roto\032\013HBase.proto\032\tWAL.proto\"[\n\024GetRegio" +
       "nInfoRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb." +
       "RegionSpecifier\022\030\n\020compaction_state\030\002 \001(" +
-      "\010\"\353\001\n\025GetRegionInfoResponse\022)\n\013region_in" +
+      "\010\"\222\002\n\025GetRegionInfoResponse\022)\n\013region_in" +
       "fo\030\001 \002(\0132\024.hbase.pb.RegionInfo\022I\n\020compac" +
       "tion_state\030\002 \001(\0162/.hbase.pb.GetRegionInf" +
       "oResponse.CompactionState\022\024\n\014isRecoverin" +
-      "g\030\003 \001(\010\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005" +
-      "MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"",
-      "P\n\023GetStoreFileRequest\022)\n\006region\030\001 \002(\0132\031" +
-      ".hbase.pb.RegionSpecifier\022\016\n\006family\030\002 \003(" +
-      "\014\"*\n\024GetStoreFileResponse\022\022\n\nstore_file\030" +
-      "\001 \003(\t\"\030\n\026GetOnlineRegionRequest\"D\n\027GetOn" +
-      "lineRegionResponse\022)\n\013region_info\030\001 \003(\0132" +
-      "\024.hbase.pb.RegionInfo\"\263\002\n\021OpenRegionRequ" +
-      "est\022=\n\topen_info\030\001 \003(\0132*.hbase.pb.OpenRe" +
-      "gionRequest.RegionOpenInfo\022\027\n\017serverStar" +
-      "tCode\030\002 \001(\004\022\032\n\022master_system_time\030\005 \001(\004\032" +
-      "\251\001\n\016RegionOpenInfo\022$\n\006region\030\001 \002(\0132\024.hba",
-      "se.pb.RegionInfo\022\037\n\027version_of_offline_n" +
-      "ode\030\002 \001(\r\022+\n\rfavored_nodes\030\003 \003(\0132\024.hbase" +
-      ".pb.ServerName\022#\n\033openForDistributedLogR" +
-      "eplay\030\004 \001(\010\"\246\001\n\022OpenRegionResponse\022F\n\rop" +
-      "ening_state\030\001 \003(\0162/.hbase.pb.OpenRegionR" +
-      "esponse.RegionOpeningState\"H\n\022RegionOpen" +
-      "ingState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPENED\020\001" +
-      "\022\022\n\016FAILED_OPENING\020\002\"?\n\023WarmupRegionRequ" +
-      "est\022(\n\nregionInfo\030\001 \002(\0132\024.hbase.pb.Regio" +
-      "nInfo\"\026\n\024WarmupRegionResponse\"\313\001\n\022CloseR",
-      "egionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb." +
-      "RegionSpecifier\022\037\n\027version_of_closing_no" +
-      "de\030\002 \001(\r\022\036\n\020transition_in_ZK\030\003 \001(\010:\004true" +
-      "\0220\n\022destination_server\030\004 \001(\0132\024.hbase.pb." +
-      "ServerName\022\027\n\017serverStartCode\030\005 \001(\004\"%\n\023C" +
-      "loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"y\n\022Fl" +
-      "ushRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase" +
-      ".pb.RegionSpecifier\022\030\n\020if_older_than_ts\030" +
-      "\002 \001(\004\022\036\n\026write_flush_wal_marker\030\003 \001(\010\"_\n" +
-      "\023FlushRegionResponse\022\027\n\017last_flush_time\030",
-      "\001 \002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026wrote_flush_wa" +
-      "l_marker\030\003 \001(\010\"T\n\022SplitRegionRequest\022)\n\006" +
-      "region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022" +
-      "\023\n\013split_point\030\002 \001(\014\"\025\n\023SplitRegionRespo" +
-      "nse\"`\n\024CompactRegionRequest\022)\n\006region\030\001 " +
-      "\002(\0132\031.hbase.pb.RegionSpecifier\022\r\n\005major\030" +
-      "\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025CompactRegionRe" +
-      "sponse\"\315\001\n\031UpdateFavoredNodesRequest\022I\n\013" +
-      "update_info\030\001 \003(\01324.hbase.pb.UpdateFavor" +
-      "edNodesRequest.RegionUpdateInfo\032e\n\020Regio",
-      "nUpdateInfo\022$\n\006region\030\001 \002(\0132\024.hbase.pb.R" +
-      "egionInfo\022+\n\rfavored_nodes\030\002 \003(\0132\024.hbase" +
-      ".pb.ServerName\".\n\032UpdateFavoredNodesResp" +
-      "onse\022\020\n\010response\030\001 \001(\r\"a\n\010WALEntry\022\035\n\003ke" +
-      "y\030\001 \002(\0132\020.hbase.pb.WALKey\022\027\n\017key_value_b" +
-      "ytes\030\002 \003(\014\022\035\n\025associated_cell_count\030\003 \001(" +
-      "\005\"\242\001\n\030ReplicateWALEntryRequest\022!\n\005entry\030" +
-      "\001 \003(\0132\022.hbase.pb.WALEntry\022\034\n\024replication" +
-      "ClusterId\030\002 \001(\t\022\"\n\032sourceBaseNamespaceDi" +
-      "rPath\030\003 \001(\t\022!\n\031sourceHFileArchiveDirPath",
-      "\030\004 \001(\t\"\033\n\031ReplicateWALEntryResponse\"\026\n\024R" +
-      "ollWALWriterRequest\"0\n\025RollWALWriterResp" +
-      "onse\022\027\n\017region_to_flush\030\001 \003(\014\"#\n\021StopSer" +
-      "verRequest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServer" +
-      "Response\"\026\n\024GetServerInfoRequest\"K\n\nServ" +
-      "erInfo\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb.S" +
-      "erverName\022\022\n\nwebui_port\030\002 \001(\r\"B\n\025GetServ" +
-      "erInfoResponse\022)\n\013server_info\030\001 \002(\0132\024.hb" +
-      "ase.pb.ServerInfo\"\034\n\032UpdateConfiguration" +
-      "Request\"\035\n\033UpdateConfigurationResponse\"?",
-      "\n\024GetRegionLoadRequest\022\'\n\ntable_name\030\001 \001" +
-      "(\0132\023.hbase.pb.TableName\"C\n\025GetRegionLoad" +
-      "Response\022*\n\014region_loads\030\001 \003(\0132\024.hbase.p" +
-      "b.RegionLoad\"\200\001\n\030ExecuteProceduresReques" +
-      "t\0220\n\013open_region\030\001 \003(\0132\033.hbase.pb.OpenRe" +
-      "gionRequest\0222\n\014close_region\030\002 \003(\0132\034.hbas" +
-      "e.pb.CloseRegionRequest\"\203\001\n\031ExecuteProce" +
-      "duresResponse\0221\n\013open_region\030\001 \003(\0132\034.hba" +
-      "se.pb.OpenRegionResponse\0223\n\014close_region" +
-      "\030\002 \003(\0132\035.hbase.pb.CloseRegionResponse\"\244\001",
-      "\n\023MergeRegionsRequest\022+\n\010region_a\030\001 \002(\0132" +
-      "\031.hbase.pb.RegionSpecifier\022+\n\010region_b\030\002" +
-      " \002(\0132\031.hbase.pb.RegionSpecifier\022\027\n\010forci" +
-      "ble\030\003 \001(\010:\005false\022\032\n\022master_system_time\030\004" +
-      " \001(\004\"\026\n\024MergeRegionsResponse2\267\014\n\014AdminSe" +
-      "rvice\022P\n\rGetRegionInfo\022\036.hbase.pb.GetReg" +
-      "ionInfoRequest\032\037.hbase.pb.GetRegionInfoR" +
-      "esponse\022M\n\014GetStoreFile\022\035.hbase.pb.GetSt" +
-      "oreFileRequest\032\036.hbase.pb.GetStoreFileRe" +
-      "sponse\022V\n\017GetOnlineRegion\022 .hbase.pb.Get",
-      "OnlineRegionRequest\032!.hbase.pb.GetOnline" +
-      "RegionResponse\022G\n\nOpenRegion\022\033.hbase.pb." +
-      "OpenRegionRequest\032\034.hbase.pb.OpenRegionR" +
-      "esponse\022M\n\014WarmupRegion\022\035.hbase.pb.Warmu" +
-      "pRegionRequest\032\036.hbase.pb.WarmupRegionRe" +
-      "sponse\022J\n\013CloseRegion\022\034.hbase.pb.CloseRe" +
-      "gionRequest\032\035.hbase.pb.CloseRegionRespon" +
-      "se\022J\n\013FlushRegion\022\034.hbase.pb.FlushRegion" +
-      "Request\032\035.hbase.pb.FlushRegionResponse\022J" +
-      "\n\013SplitRegion\022\034.hbase.pb.SplitRegionRequ",
-      "est\032\035.hbase.pb.SplitRegionResponse\022P\n\rCo" +
-      "mpactRegion\022\036.hbase.pb.CompactRegionRequ" +
-      "est\032\037.hbase.pb.CompactRegionResponse\022\\\n\021" +
-      "ReplicateWALEntry\022\".hbase.pb.ReplicateWA" +
-      "LEntryRequest\032#.hbase.pb.ReplicateWALEnt" +
-      "ryResponse\022Q\n\006Replay\022\".hbase.pb.Replicat" +
-      "eWALEntryRequest\032#.hbase.pb.ReplicateWAL" +
-      "EntryResponse\022P\n\rRollWALWriter\022\036.hbase.p" +
-      "b.RollWALWriterRequest\032\037.hbase.pb.RollWA" +
-      "LWriterResponse\022P\n\rGetServerInfo\022\036.hbase",
-      ".pb.GetServerInfoRequest\032\037.hbase.pb.GetS" +
-      "erverInfoResponse\022G\n\nStopServer\022\033.hbase." +
-      "pb.StopServerRequest\032\034.hbase.pb.StopServ" +
-      "erResponse\022_\n\022UpdateFavoredNodes\022#.hbase" +
-      ".pb.UpdateFavoredNodesRequest\032$.hbase.pb" +
-      ".UpdateFavoredNodesResponse\022b\n\023UpdateCon" +
-      "figuration\022$.hbase.pb.UpdateConfiguratio" +
-      "nRequest\032%.hbase.pb.UpdateConfigurationR" +
-      "esponse\022P\n\rGetRegionLoad\022\036.hbase.pb.GetR" +
-      "egionLoadRequest\032\037.hbase.pb.GetRegionLoa",
-      "dResponse\022\\\n\021ExecuteProcedures\022\".hbase.p" +
-      "b.ExecuteProceduresRequest\032#.hbase.pb.Ex" +
-      "ecuteProceduresResponse\022M\n\014MergeRegions\022" +
-      "\035.hbase.pb.MergeRegionsRequest\032\036.hbase.p" +
-      "b.MergeRegionsResponseBH\n1org.apache.had" +
-      "oop.hbase.shaded.protobuf.generatedB\013Adm" +
-      "inProtosH\001\210\001\001\240\001\001"
+      "g\030\003 \001(\010\022\022\n\nsplittable\030\004 \001(\010\022\021\n\tmergeable" +
+      "\030\005 \001(\010\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005M",
+      "INOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"P" +
+      "\n\023GetStoreFileRequest\022)\n\006region\030\001 \002(\0132\031." +
+      "hbase.pb.RegionSpecifier\022\016\n\006family\030\002 \003(\014" +
+      "\"*\n\024GetStoreFileResponse\022\022\n\nstore_file\030\001" +
+      " \003(\t\"\030\n\026GetOnlineRegionRequest\"D\n\027GetOnl" +
+      "ineRegionResponse\022)\n\013region_info\030\001 \003(\0132\024" +
+      ".hbase.pb.RegionInfo\"\263\002\n\021OpenRegionReque" +
+      "st\022=\n\topen_info\030\001 \003(\0132*.hbase.pb.OpenReg" +
+      "ionRequest.RegionOpenInfo\022\027\n\017serverStart" +
+      "Code\030\002 \001(\004\022\032\n\022master_system_time\030\005 \001(\004\032\251",
+      "\001\n\016RegionOpenInfo\022$\n\006region\030\001 \002(\0132\024.hbas" +
+      "e.pb.RegionInfo\022\037\n\027version_of_offline_no" +
+      "de\030\002 \001(\r\022+\n\rfavored_nodes\030\003 \003(\0132\024.hbase." +
+      "pb.ServerName\022#\n\033openForDistributedLogRe" +
+      "play\030\004 \001(\010\"\246\001\n\022OpenRegionResponse\022F\n\rope" +
+      "ning_state\030\001 \003(\0162/.hbase.pb.OpenRegionRe" +
+      "sponse.RegionOpeningState\"H\n\022RegionOpeni" +
+      "ngState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPENED\020\001\022" +
+      "\022\n\016FAILED_OPENING\020\002\"?\n\023WarmupRegionReque" +
+      "st\022(\n\nregionInfo\030\001 \002(\0132\024.hbase.pb.Region",
+      "Info\"\026\n\024WarmupRegionResponse\"\313\001\n\022CloseRe" +
+      "gionRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.R" +
+      "egionSpecifier\022\037\n\027version_of_closing_nod" +
+      "e\030\002 \001(\r\022\036\n\020transition_in_ZK\030\003 \001(\010:\004true\022" +
+      "0\n\022destination_server\030\004 \001(\0132\024.hbase.pb.S" +
+      "erverName\022\027\n\017serverStartCode\030\005 \001(\004\"%\n\023Cl" +
+      "oseRegionResponse\022\016\n\006closed\030\001 \002(\010\"y\n\022Flu" +
+      "shRegionRequest\022)\n\006region\030\001 \002(\0132\031.hbase." +
+      "pb.RegionSpecifier\022\030\n\020if_older_than_ts\030\002" +
+      " \001(\004\022\036\n\026write_flush_wal_marker\030\003 \001(\010\"_\n\023",
+      "FlushRegionResponse\022\027\n\017last_flush_time\030\001" +
+      " \002(\004\022\017\n\007flushed\030\002 \001(\010\022\036\n\026wrote_flush_wal" +
+      "_marker\030\003 \001(\010\"T\n\022SplitRegionRequest\022)\n\006r" +
+      "egion\030\001 \002(\0132\031.hbase.pb.RegionSpecifier\022\023" +
+      "\n\013split_point\030\002 \001(\014\"\025\n\023SplitRegionRespon" +
+      "se\"`\n\024CompactRegionRequest\022)\n\006region\030\001 \002" +
+      "(\0132\031.hbase.pb.RegionSpecifier\022\r\n\005major\030\002" +
+      " \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025CompactRegionRes" +
+      "ponse\"\315\001\n\031UpdateFavoredNodesRequest\022I\n\013u" +
+      "pdate_info\030\001 \003(\01324.hbase.pb.UpdateFavore",
+      "dNodesRequest.RegionUpdateInfo\032e\n\020Region" +
+      "UpdateInfo\022$\n\006region\030\001 \002(\0132\024.hbase.pb.Re" +
+      "gionInfo\022+\n\rfavored_nodes\030\002 \003(\0132\024.hbase." +
+      "pb.ServerName\".\n\032UpdateFavoredNodesRespo" +
+      "nse\022\020\n\010response\030\001 \001(\r\"a\n\010WALEntry\022\035\n\003key" +
+      "\030\001 \002(\0132\020.hbase.pb.WALKey\022\027\n\017key_value_by" +
+      "tes\030\002 \003(\014\022\035\n\025associated_cell_count\030\003 \001(\005" +
+      "\"\242\001\n\030ReplicateWALEntryRequest\022!\n\005entry\030\001" +
+      " \003(\0132\022.hbase.pb.WALEntry\022\034\n\024replicationC" +
+      "lusterId\030\002 \001(\t\022\"\n\032sourceBaseNamespaceDir",
+      "Path\030\003 \001(\t\022!\n\031sourceHFileArchiveDirPath\030" +
+      "\004 \001(\t\"\033\n\031ReplicateWALEntryResponse\"\026\n\024Ro" +
+      "llWALWriterRequest\"0\n\025RollWALWriterRespo" +
+      "nse\022\027\n\017region_to_flush\030\001 \003(\014\"#\n\021StopServ" +
+      "erRequest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerR" +
+      "esponse\"\026\n\024GetServerInfoRequest\"K\n\nServe" +
+      "rInfo\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb.Se" +
+      "rverName\022\022\n\nwebui_port\030\002 \001(\r\"B\n\025GetServe" +
+      "rInfoResponse\022)\n\013server_info\030\001 \002(\0132\024.hba" +
+      "se.pb.ServerInfo\"\034\n\032UpdateConfigurationR",
+      "equest\"\035\n\033UpdateConfigurationResponse\"?\n" +
+      "\024GetRegionLoadRequest\022\'\n\ntable_name\030\001 \001(" +
+      "\0132\023.hbase.pb.TableName\"C\n\025GetRegionLoadR" +
+      "esponse\022*\n\014region_loads\030\001 \003(\0132\024.hbase.pb" +
+      ".RegionLoad\"\200\001\n\030ExecuteProceduresRequest" +
+      "\0220\n\013open_region\030\001 \003(\0132\033.hbase.pb.OpenReg" +
+      "ionRequest\0222\n\014close_region\030\002 \003(\0132\034.hbase" +
+      ".pb.CloseRegionRequest\"\203\001\n\031ExecuteProced" +
+      "uresResponse\0221\n\013open_region\030\001 \003(\0132\034.hbas" +
+      "e.pb.OpenRegionResponse\0223\n\014close_region\030",
+      "\002 \003(\0132\035.hbase.pb.CloseRegionResponse\"\244\001\n" +
+      "\023MergeRegionsRequest\022+\n\010region_a\030\001 \002(\0132\031" +
+      ".hbase.pb.RegionSpecifier\022+\n\010region_b\030\002 " +
+      "\002(\0132\031.hbase.pb.RegionSpecifier\022\027\n\010forcib" +
+      "le\030\003 \001(\010:\005false\022\032\n\022master_system_time\030\004 " +
+      "\001(\004\"\026\n\024MergeRegionsResponse2\267\014\n\014AdminSer" +
+      "vice\022P\n\rGetRegionInfo\022\036.hbase.pb.GetRegi" +
+      "onInfoRequest\032\037.hbase.pb.GetRegionInfoRe" +
+      "sponse\022M\n\014GetStoreFile\022\035.hbase.pb.GetSto" +
+      "reFileRequest\032\036.hbase.pb.GetStoreFileRes",
+      "ponse\022V\n\017GetOnlineRegion\022 .hbase.pb.GetO" +
+      "nlineRegionRequest\032!.hbase.pb.GetOnlineR" +
+      "egionResponse\022G\n\nOpenRegion\022\033.hbase.pb.O" +
+      "penRegionRequest\032\034.hbase.pb.OpenRegionRe" +
+      "sponse\022M\n\014WarmupRegion\022\035.hbase.pb.Warmup" +
+      "RegionRequest\032\036.hbase.pb.WarmupRegionRes" +
+      "ponse\022J\n\013CloseRegion\022\034.hbase.pb.CloseReg" +
+      "ionRequest\032\035.hbase.pb.CloseRegionRespons" +
+      "e\022J\n\013FlushRegion\022\034.hbase.pb.FlushRegionR" +
+      "equest\032\035.hbase.pb.FlushRegionResponse\022J\n",
+      "\013SplitRegion\022\034.hbase.pb.SplitRegionReque" +
+      "st\032\035.hbase.pb.SplitRegionResponse\022P\n\rCom" +
+      "pactRegion\022\036.hbase.pb.CompactRegionReque" +
+      "st\032\037.hbase.pb.CompactRegionResponse\022\\\n\021R" +
+      "eplicateWALEntry\022\".hbase.pb.ReplicateWAL" +
+      "EntryRequest\032#.hbase.pb.ReplicateWALEntr" +
+      "yResponse\022Q\n\006Replay\022\".hbase.pb.Replicate" +
+      "WALEntryRequest\032#.hbase.pb.ReplicateWALE" +
+      "ntryResponse\022P\n\rRollWALWriter\022\036.hbase.pb" +
+      ".RollWALWriterRequest\032\037.hbase.pb.RollWAL",
+      "WriterResponse\022P\n\rGetServerInfo\022\036.hbase." +
+      "pb.GetServerInfoRequest\032\037.hbase.pb.GetSe" +
+      "rverInfoResponse\022G\n\nStopServer\022\033.hbase.p" +
+      "b.StopServerRequest\032\034.hbase.pb.StopServe" +
+      "rResponse\022_\n\022UpdateFavoredNodes\022#.hbase." +
+      "pb.UpdateFavoredNodesRequest\032$.hbase.pb." +
+      "UpdateFavoredNodesResponse\022b\n\023UpdateConf" +
+      "iguration\022$.hbase.pb.UpdateConfiguration" +
+      "Request\032%.hbase.pb.UpdateConfigurationRe" +
+      "sponse\022P\n\rGetRegionLoad\022\036.hbase.pb.GetRe",
+      "gionLoadRequest\032\037.hbase.pb.GetRegionLoad" +
+      "Response\022\\\n\021ExecuteProcedures\022\".hbase.pb" +
+      ".ExecuteProceduresRequest\032#.hbase.pb.Exe" +
+      "cuteProceduresResponse\022M\n\014MergeRegions\022\035" +
+      ".hbase.pb.MergeRegionsRequest\032\036.hbase.pb" +
+      ".MergeRegionsResponseBH\n1org.apache.hado" +
+      "op.hbase.shaded.protobuf.generatedB\013Admi" +
+      "nProtosH\001\210\001\001\240\001\001"
     };
     org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
         new org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FileDescriptor.    InternalDescriptorAssigner() {
@@ -29036,7 +29277,7 @@ public final class AdminProtos {
     internal_static_hbase_pb_GetRegionInfoResponse_fieldAccessorTable = new
       org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable(
         internal_static_hbase_pb_GetRegionInfoResponse_descriptor,
-        new java.lang.String[] { "RegionInfo", "CompactionState", "IsRecovering", });
+        new java.lang.String[] { "RegionInfo", "CompactionState", "IsRecovering", "Splittable", "Mergeable", });
     internal_static_hbase_pb_GetStoreFileRequest_descriptor =
       getDescriptor().getMessageTypes().get(2);
     internal_static_hbase_pb_GetStoreFileRequest_fieldAccessorTable = new