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

[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.

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