You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2015/04/09 23:49:06 UTC

[15/18] hbase git commit: HBASE-13211 Procedure V2 - master Enable/Disable table (Stephen Yuan Jiang)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-protocol/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/MasterProcedure.proto b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
index f09dc98..012b279 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -139,3 +139,33 @@ message DeleteColumnFamilyMessage {
   optional TableSchema unmodified_table_schema = 4;
   repeated RegionInfo region_info = 5;
 }
+
+enum EnableTableState {
+  ENABLE_TABLE_PREPARE = 1;
+  ENABLE_TABLE_PRE_OPERATION = 2;
+  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
+  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
+  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
+  ENABLE_TABLE_POST_OPERATION = 6;
+}
+
+message EnableTableMessage {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+enum DisableTableState {
+  DISABLE_TABLE_PREPARE = 1;
+  DISABLE_TABLE_PRE_OPERATION = 2;
+  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
+  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
+  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
+  DISABLE_TABLE_POST_OPERATION = 6;
+}
+
+message DisableTableMessage {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 2e33095..ff28081 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -87,14 +87,14 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
-import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
-import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
 import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
@@ -1681,11 +1681,24 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.preEnableTable(tableName);
     }
     LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
-    this.service.submit(new EnableTableHandler(this, tableName,
-      assignmentManager, tableLockManager, false).prepare());
+
+    // Execute the operation asynchronously - client will check the progress of the operation
+    final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
+    long procId =
+        this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor
+            .getEnvironment(), tableName, false, prepareLatch));
+    // Before returning to client, we want to make sure that the table is prepared to be
+    // enabled (the table is locked and the table state is set).
+    //
+    // Note: if the procedure throws exception, we will catch it and rethrow.
+    prepareLatch.await();
+
     if (cpHost != null) {
       cpHost.postEnableTable(tableName);
-   }
+    }
+
+    // TODO: return procId as part of client-side change
+    // return procId;
   }
 
   @Override
@@ -1695,11 +1708,25 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.preDisableTable(tableName);
     }
     LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
-    this.service.submit(new DisableTableHandler(this, tableName,
-      assignmentManager, tableLockManager, false).prepare());
+
+    // Execute the operation asynchronously - client will check the progress of the operation
+    final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
+    // Execute the operation asynchronously - client will check the progress of the operation
+    long procId =
+        this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor
+            .getEnvironment(), tableName, false, prepareLatch));
+    // Before returning to client, we want to make sure that the table is prepared to be
+    // enabled (the table is locked and the table state is set).
+    //
+    // Note: if the procedure throws exception, we will catch it and rethrow.
+    prepareLatch.await();
+
     if (cpHost != null) {
       cpHost.postDisableTable(tableName);
     }
+
+    // TODO: return procId as part of client-side change
+    // return procId;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
new file mode 100644
index 0000000..870ad75
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java
@@ -0,0 +1,542 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.BulkAssigner;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.htrace.Trace;
+
+@InterfaceAudience.Private
+public class DisableTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, DisableTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(DisableTableProcedure.class);
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private TableName tableName;
+  private boolean skipTableStateCheck;
+  private UserGroupInformation user;
+
+  private Boolean traceEnabled = null;
+  // This is for back compatible with 1.0 asynchronized operations.
+  private final ProcedurePrepareLatch syncLatch;
+
+  enum MarkRegionOfflineOpResult {
+    MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL,
+    BULK_ASSIGN_REGIONS_FAILED,
+    MARK_ALL_REGIONS_OFFLINE_INTERRUPTED,
+  }
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   */
+  public DisableTableProcedure() {
+    syncLatch = null;
+  }
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   * @param tableName the table to operate on
+   * @param skipTableStateCheck whether to check table state
+   * @throws IOException
+   */
+  public DisableTableProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final boolean skipTableStateCheck) throws IOException {
+    this(env, tableName, skipTableStateCheck, null);
+  }
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   * @param tableName the table to operate on
+   * @param skipTableStateCheck whether to check table state
+   * @throws IOException
+   */
+  public DisableTableProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final boolean skipTableStateCheck,
+      final ProcedurePrepareLatch syncLatch) throws IOException {
+    this.tableName = tableName;
+    this.skipTableStateCheck = skipTableStateCheck;
+    this.user = env.getRequestUser().getUGI();
+
+    // Compatible with 1.0: We use latch to make sure that this procedure implementation is
+    // compatible with 1.0 asynchronized operations. We need to lock the table and check
+    // whether the Disable operation could be performed (table exists and online; table state
+    // is ENABLED). Once it is done, we are good to release the latch and the client can
+    // start asynchronously wait for the operation.
+    //
+    // Note: the member syncLatch could be null if we are in failover or recovery scenario.
+    // This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
+    this.syncLatch = syncLatch;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final DisableTableState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case DISABLE_TABLE_PREPARE:
+        if (prepareDisable(env)) {
+          setNextState(DisableTableState.DISABLE_TABLE_PRE_OPERATION);
+        } else {
+          assert isFailed() : "disable should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
+        break;
+      case DISABLE_TABLE_PRE_OPERATION:
+        preDisable(env, state);
+        setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLING_TABLE_STATE);
+        break;
+      case DISABLE_TABLE_SET_DISABLING_TABLE_STATE:
+        setTableStateToDisabling(env, tableName);
+        setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
+        break;
+      case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
+        if (markRegionsOffline(env, tableName, true) ==
+            MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
+          setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
+        } else {
+          LOG.trace("Retrying later to disable the missing regions");
+        }
+        break;
+      case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
+        setTableStateToDisabled(env, tableName);
+        setNextState(DisableTableState.DISABLE_TABLE_POST_OPERATION);
+        break;
+      case DISABLE_TABLE_POST_OPERATION:
+        postDisable(env, state);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.warn("Retriable error trying to disable table=" + tableName + " state=" + state, e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final DisableTableState state)
+      throws IOException {
+    if (state == DisableTableState.DISABLE_TABLE_PREPARE) {
+      // nothing to rollback, prepare-disable is just table-state checks.
+      // We can fail if the table does not exist or is not disabled.
+      ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+      return;
+    }
+
+    // The delete doesn't have a rollback. The execution will succeed, at some point.
+    throw new UnsupportedOperationException("unhandled state=" + state);
+  }
+
+  @Override
+  protected DisableTableState getState(final int stateId) {
+    return DisableTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final DisableTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected DisableTableState getInitialState() {
+    return DisableTableState.DISABLE_TABLE_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(final DisableTableState state) {
+    if (aborted.get()) {
+      setAbortFailure("disable-table", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      tableName,
+      EventType.C_M_DISABLE_TABLE.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(tableName);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.DisableTableMessage.Builder disableTableMsg =
+        MasterProcedureProtos.DisableTableMessage.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setSkipTableStateCheck(skipTableStateCheck);
+
+    disableTableMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.DisableTableMessage disableTableMsg =
+        MasterProcedureProtos.DisableTableMessage.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
+    skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.DISABLE;
+  }
+
+  /**
+   * Action before any real action of disabling table. Set the exception in the procedure instead
+   * of throwing it.  This approach is to deal with backward compatible with 1.0.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
+    boolean canTableBeDisabled = true;
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      setFailure("master-disable-table", new ConstraintException("Cannot disable catalog table"));
+      canTableBeDisabled = false;
+    } else if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
+      setFailure("master-disable-table", new TableNotFoundException(tableName));
+      canTableBeDisabled = false;
+    } else if (!skipTableStateCheck) {
+      // There could be multiple client requests trying to disable or enable
+      // the table at the same time. Ensure only the first request is honored
+      // After that, no other requests can be accepted until the table reaches
+      // DISABLED or ENABLED.
+      //
+      // Note: in 1.0 release, we called TableStateManager.setTableStateIfInStates() to set
+      // the state to DISABLING from ENABLED. The implementation was done before table lock
+      // was implemented. With table lock, there is no need to set the state here (it will
+      // set the state later on). A quick state check should be enough for us to move forward.
+      TableStateManager tsm =
+        env.getMasterServices().getAssignmentManager().getTableStateManager();
+      if (!tsm.getTableState(tableName).equals(TableState.State.ENABLED)) {
+        LOG.info("Table " + tableName + " isn't enabled; skipping disable");
+        setFailure("master-disable-table", new TableNotEnabledException(tableName));
+        canTableBeDisabled = false;
+      }
+    }
+
+    // We are done the check. Future actions in this procedure could be done asynchronously.
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+
+    return canTableBeDisabled;
+  }
+
+  /**
+   * Action before disabling table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected void preDisable(final MasterProcedureEnv env, final DisableTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Mark table state to Disabling
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  protected static void setTableStateToDisabling(
+      final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    // Set table disabling flag up in zk.
+    env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
+      tableName,
+      TableState.State.DISABLING);
+  }
+
+  /**
+   * Mark regions of the table offline with retries
+   * @param env MasterProcedureEnv
+   * @param tableName the target table
+   * @param retryRequired whether to retry if the first run failed
+   * @return whether the operation is fully completed or being interrupted.
+   * @throws IOException
+   */
+  protected static MarkRegionOfflineOpResult markRegionsOffline(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final Boolean retryRequired) throws IOException {
+    // Dev consideration: add a config to control max number of retry. For now, it is hard coded.
+    int maxTry = (retryRequired ? 10 : 1);
+    MarkRegionOfflineOpResult operationResult =
+        MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
+    do {
+      try {
+        operationResult = markRegionsOffline(env, tableName);
+        if (operationResult == MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
+          break;
+        }
+        maxTry--;
+      } catch (Exception e) {
+        LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
+        maxTry--;
+        if (maxTry > 0) {
+          continue; // we still have some retry left, try again.
+        }
+        throw e;
+      }
+    } while (maxTry > 0);
+
+    if (operationResult != MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
+      LOG.warn("Some or all regions of the Table '" + tableName + "' were still online");
+    }
+
+    return operationResult;
+  }
+
+  /**
+   * Mark regions of the table offline
+   * @param env MasterProcedureEnv
+   * @param tableName the target table
+   * @return whether the operation is fully completed or being interrupted.
+   * @throws IOException
+   */
+  private static MarkRegionOfflineOpResult markRegionsOffline(
+      final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    // Get list of online regions that are of this table.  Regions that are
+    // already closed will not be included in this list; i.e. the returned
+    // list is not ALL regions in a table, its all online regions according
+    // to the in-memory state on this master.
+    MarkRegionOfflineOpResult operationResult =
+        MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL;
+    final List<HRegionInfo> regions =
+        env.getMasterServices().getAssignmentManager().getRegionStates()
+            .getRegionsOfTable(tableName);
+    if (regions.size() > 0) {
+      LOG.info("Offlining " + regions.size() + " regions.");
+
+      BulkDisabler bd = new BulkDisabler(env, tableName, regions);
+      try {
+        if (!bd.bulkAssign()) {
+          operationResult = MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Disable was interrupted");
+        // Preserve the interrupt.
+        Thread.currentThread().interrupt();
+        operationResult = MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_INTERRUPTED;
+      }
+    }
+    return operationResult;
+  }
+
+  /**
+   * Mark table state to Disabled
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  protected static void setTableStateToDisabled(
+      final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    // Flip the table to disabled
+    env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
+      tableName,
+      TableState.State.DISABLED);
+    LOG.info("Disabled table, " + tableName + ", is completed.");
+  }
+
+  /**
+   * Action after disabling table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  protected void postDisable(final MasterProcedureEnv env, final DisableTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+
+  /**
+   * Coprocessor Action.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void runCoprocessorAction(final MasterProcedureEnv env, final DisableTableState state)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          switch (state) {
+          case DISABLE_TABLE_PRE_OPERATION:
+            cpHost.preDisableTableHandler(tableName);
+            break;
+          case DISABLE_TABLE_POST_OPERATION:
+            cpHost.postDisableTableHandler(tableName);
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+
+  /**
+   * Run bulk disable.
+   */
+  private static class BulkDisabler extends BulkAssigner {
+    private final AssignmentManager assignmentManager;
+    private final List<HRegionInfo> regions;
+    private final TableName tableName;
+    private final int waitingTimeForEvents;
+
+    public BulkDisabler(final MasterProcedureEnv env, final TableName tableName,
+        final List<HRegionInfo> regions) {
+      super(env.getMasterServices());
+      this.assignmentManager = env.getMasterServices().getAssignmentManager();
+      this.tableName = tableName;
+      this.regions = regions;
+      this.waitingTimeForEvents =
+          env.getMasterServices().getConfiguration()
+              .getInt("hbase.master.event.waiting.time", 1000);
+    }
+
+    @Override
+    protected void populatePool(ExecutorService pool) {
+      RegionStates regionStates = assignmentManager.getRegionStates();
+      for (final HRegionInfo region : regions) {
+        if (regionStates.isRegionInTransition(region)
+            && !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) {
+          continue;
+        }
+        pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler", new Runnable() {
+          @Override
+          public void run() {
+            assignmentManager.unassign(region);
+          }
+        }));
+      }
+    }
+
+    @Override
+    protected boolean waitUntilDone(long timeout) throws InterruptedException {
+      long startTime = EnvironmentEdgeManager.currentTime();
+      long remaining = timeout;
+      List<HRegionInfo> regions = null;
+      long lastLogTime = startTime;
+      while (!server.isStopped() && remaining > 0) {
+        Thread.sleep(waitingTimeForEvents);
+        regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
+        long now = EnvironmentEdgeManager.currentTime();
+        // Don't log more than once every ten seconds. Its obnoxious. And only log table regions
+        // if we are waiting a while for them to go down...
+        if (LOG.isDebugEnabled() && ((now - lastLogTime) > 10000)) {
+          lastLogTime = now;
+          LOG.debug("Disable waiting until done; " + remaining + " ms remaining; " + regions);
+        }
+        if (regions.isEmpty()) break;
+        remaining = timeout - (now - startTime);
+      }
+      return regions != null && regions.isEmpty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
new file mode 100644
index 0000000..770e207
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java
@@ -0,0 +1,586 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.BulkAssigner;
+import org.apache.hadoop.hbase.master.GeneralBulkAssigner;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+public class EnableTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, EnableTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(EnableTableProcedure.class);
+
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+  private TableName tableName;
+  private boolean skipTableStateCheck;
+  private UserGroupInformation user;
+
+  private Boolean traceEnabled = null;
+  // This is for back compatible with 1.0 asynchronized operations.
+  private final ProcedurePrepareLatch syncLatch;
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  public EnableTableProcedure() {
+    syncLatch = null;
+  }
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   * @param tableName the table to operate on
+   * @param skipTableStateCheck whether to check table state
+   * @throws IOException
+   */
+  public EnableTableProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final boolean skipTableStateCheck) throws IOException {
+    this(env, tableName, skipTableStateCheck, null);
+  }
+
+  /**
+   * Constructor
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   * @param tableName the table to operate on
+   * @param skipTableStateCheck whether to check table state
+   */
+  public EnableTableProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final boolean skipTableStateCheck,
+      final ProcedurePrepareLatch syncLatch) throws IOException {
+    this.tableName = tableName;
+    this.skipTableStateCheck = skipTableStateCheck;
+    this.user = env.getRequestUser().getUGI();
+
+    // Compatible with 1.0: We use latch to make sure that this procedure implementation is
+    // compatible with 1.0 asynchronized operations. We need to lock the table and check
+    // whether the Enable operation could be performed (table exists and offline; table state
+    // is DISABLED). Once it is done, we are good to release the latch and the client can
+    // start asynchronously wait for the operation.
+    //
+    // Note: the member syncLatch could be null if we are in failover or recovery scenario.
+    // This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
+    this.syncLatch = syncLatch;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final EnableTableState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case ENABLE_TABLE_PREPARE:
+        if (prepareEnable(env)) {
+          setNextState(EnableTableState.ENABLE_TABLE_PRE_OPERATION);
+        } else {
+          assert isFailed() : "enable should have an exception here";
+          return Flow.NO_MORE_STATE;
+        }
+        break;
+      case ENABLE_TABLE_PRE_OPERATION:
+        preEnable(env, state);
+        setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLING_TABLE_STATE);
+        break;
+      case ENABLE_TABLE_SET_ENABLING_TABLE_STATE:
+        setTableStateToEnabling(env, tableName);
+        setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
+        break;
+      case ENABLE_TABLE_MARK_REGIONS_ONLINE:
+        markRegionsOnline(env, tableName, true);
+        setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLED_TABLE_STATE);
+        break;
+      case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
+        setTableStateToEnabled(env, tableName);
+        setNextState(EnableTableState.ENABLE_TABLE_POST_OPERATION);
+        break;
+      case ENABLE_TABLE_POST_OPERATION:
+        postEnable(env, state);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.error("Error trying to enable table=" + tableName + " state=" + state, e);
+      setFailure("master-enable-table", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final EnableTableState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case ENABLE_TABLE_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo (eg. DisableTableProcedure.preDisable())?
+        break;
+      case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
+        DisableTableProcedure.setTableStateToDisabling(env, tableName);
+        break;
+      case ENABLE_TABLE_MARK_REGIONS_ONLINE:
+        markRegionsOfflineDuringRecovery(env);
+        break;
+      case ENABLE_TABLE_SET_ENABLING_TABLE_STATE:
+        DisableTableProcedure.setTableStateToDisabled(env, tableName);
+        break;
+      case ENABLE_TABLE_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo (eg. DisableTableProcedure.postDisable())?
+        break;
+      case ENABLE_TABLE_PREPARE:
+        // Nothing to undo for this state.
+        // We do need to count down the latch count so that we don't stuck.
+        ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+        break;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed enable table rollback attempt step=" + state + " table=" + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected EnableTableState getState(final int stateId) {
+    return EnableTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final EnableTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected EnableTableState getInitialState() {
+    return EnableTableState.ENABLE_TABLE_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(final EnableTableState state) {
+    if (aborted.get()) {
+      setAbortFailure("Enable-table", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      tableName,
+      EventType.C_M_ENABLE_TABLE.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(tableName);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.EnableTableMessage.Builder enableTableMsg =
+        MasterProcedureProtos.EnableTableMessage.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setSkipTableStateCheck(skipTableStateCheck);
+
+    enableTableMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.EnableTableMessage enableTableMsg =
+        MasterProcedureProtos.EnableTableMessage.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(enableTableMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(enableTableMsg.getTableName());
+    skipTableStateCheck = enableTableMsg.getSkipTableStateCheck();
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.ENABLE;
+  }
+
+
+  /**
+   * Action before any real action of enabling table. Set the exception in the procedure instead
+   * of throwing it.  This approach is to deal with backward compatible with 1.0.
+   * @param env MasterProcedureEnv
+   * @return whether the table passes the necessary checks
+   * @throws IOException
+   */
+  private boolean prepareEnable(final MasterProcedureEnv env) throws IOException {
+    boolean canTableBeEnabled = true;
+
+    // Check whether table exists
+    if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
+      setFailure("master-enable-table", new TableNotFoundException(tableName));
+      canTableBeEnabled = false;
+    } else if (!skipTableStateCheck) {
+      // There could be multiple client requests trying to disable or enable
+      // the table at the same time. Ensure only the first request is honored
+      // After that, no other requests can be accepted until the table reaches
+      // DISABLED or ENABLED.
+      //
+      // Note: in 1.0 release, we called TableStateManager.setTableStateIfInStates() to set
+      // the state to ENABLING from DISABLED. The implementation was done before table lock
+      // was implemented. With table lock, there is no need to set the state here (it will
+      // set the state later on). A quick state check should be enough for us to move forward.
+      TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
+      if (!tsm.getTableState(tableName).equals(TableState.State.DISABLED)) {
+        LOG.info("Table " + tableName + " isn't disabled; skipping enable");
+        setFailure("master-enable-table", new TableNotDisabledException(this.tableName));
+        canTableBeEnabled = false;
+      }
+    }
+
+    // We are done the check. Future actions in this procedure could be done asynchronously.
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+
+    return canTableBeEnabled;
+  }
+
+  /**
+   * Action before enabling table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preEnable(final MasterProcedureEnv env, final EnableTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Mark table state to Enabling
+   * @param env MasterProcedureEnv
+   * @param tableName the target table
+   * @throws IOException
+   */
+  protected static void setTableStateToEnabling(
+      final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    // Set table disabling flag up in zk.
+    LOG.info("Attempting to enable the table " + tableName);
+    env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
+      tableName,
+      TableState.State.ENABLING);
+  }
+
+  /**
+   * Mark offline regions of the table online with retry
+   * @param env MasterProcedureEnv
+   * @param tableName the target table
+   * @param retryRequired whether to retry if the first run failed
+   * @return whether the operation is fully completed or being interrupted.
+   * @throws IOException
+   */
+  protected static void markRegionsOnline(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final Boolean retryRequired) throws IOException {
+    // This is best effort approach to make all regions of a table online.  If we fail to do
+    // that, it is ok that the table has some offline regions; user can fix it manually.
+
+    // Dev consideration: add a config to control max number of retry. For now, it is hard coded.
+    int maxTry = (retryRequired ? 10 : 1);
+    boolean done = false;
+
+    do {
+      try {
+        done = markRegionsOnline(env, tableName);
+        if (done) {
+          break;
+        }
+        maxTry--;
+      } catch (Exception e) {
+        LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
+        maxTry--;
+        if (maxTry > 0) {
+          continue; // we still have some retry left, try again.
+        }
+        throw e;
+      }
+    } while (maxTry > 0);
+
+    if (!done) {
+      LOG.warn("Some or all regions of the Table '" + tableName + "' were offline");
+    }
+  }
+
+  /**
+   * Mark offline regions of the table online
+   * @param env MasterProcedureEnv
+   * @param tableName the target table
+   * @return whether the operation is fully completed or being interrupted.
+   * @throws IOException
+   */
+  private static boolean markRegionsOnline(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
+    final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
+    final MasterServices masterServices = env.getMasterServices();
+    final ServerManager serverManager = masterServices.getServerManager();
+    boolean done = false;
+    // Get the regions of this table. We're done when all listed
+    // tables are onlined.
+    List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations;
+
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      tableRegionsAndLocations =
+          new MetaTableLocator().getMetaRegionsAndLocations(masterServices.getZooKeeper());
+    } else {
+      tableRegionsAndLocations =
+          MetaTableAccessor.getTableRegionsAndLocations(masterServices.getConnection(), tableName);
+    }
+
+    int countOfRegionsInTable = tableRegionsAndLocations.size();
+    Map<HRegionInfo, ServerName> regionsToAssign =
+        regionsToAssignWithServerName(env, tableRegionsAndLocations);
+
+    // need to potentially create some regions for the replicas
+    List<HRegionInfo> unrecordedReplicas =
+        AssignmentManager.replicaRegionsNotRecordedInMeta(new HashSet<HRegionInfo>(
+            regionsToAssign.keySet()), masterServices);
+    Map<ServerName, List<HRegionInfo>> srvToUnassignedRegs =
+        assignmentManager.getBalancer().roundRobinAssignment(unrecordedReplicas,
+          serverManager.getOnlineServersList());
+    if (srvToUnassignedRegs != null) {
+      for (Map.Entry<ServerName, List<HRegionInfo>> entry : srvToUnassignedRegs.entrySet()) {
+        for (HRegionInfo h : entry.getValue()) {
+          regionsToAssign.put(h, entry.getKey());
+        }
+      }
+    }
+
+    int offlineRegionsCount = regionsToAssign.size();
+
+    LOG.info("Table '" + tableName + "' has " + countOfRegionsInTable + " regions, of which "
+        + offlineRegionsCount + " are offline.");
+    if (offlineRegionsCount == 0) {
+      return true;
+    }
+
+    List<ServerName> onlineServers = serverManager.createDestinationServersList();
+    Map<ServerName, List<HRegionInfo>> bulkPlan =
+        env.getMasterServices().getAssignmentManager().getBalancer()
+            .retainAssignment(regionsToAssign, onlineServers);
+    if (bulkPlan != null) {
+      LOG.info("Bulk assigning " + offlineRegionsCount + " region(s) across " + bulkPlan.size()
+          + " server(s), retainAssignment=true");
+
+      BulkAssigner ba = new GeneralBulkAssigner(masterServices, bulkPlan, assignmentManager, true);
+      try {
+        if (ba.bulkAssign()) {
+          done = true;
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Enable operation was interrupted when enabling table '" + tableName + "'");
+        // Preserve the interrupt.
+        Thread.currentThread().interrupt();
+      }
+    } else {
+      LOG.info("Balancer was unable to find suitable servers for table " + tableName
+          + ", leaving unassigned");
+    }
+    return done;
+  }
+
+  /**
+   * Mark regions of the table offline during recovery
+   * @param env MasterProcedureEnv
+   */
+  private void markRegionsOfflineDuringRecovery(final MasterProcedureEnv env) {
+    try {
+      // This is a best effort attempt. We will move on even it does not succeed. We will retry
+      // several times until we giving up.
+      DisableTableProcedure.markRegionsOffline(env, tableName, true);
+    } catch (Exception e) {
+      LOG.debug("Failed to offline all regions of table " + tableName + ". Ignoring", e);
+    }
+  }
+
+  /**
+   * Mark table state to Enabled
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  protected static void setTableStateToEnabled(
+      final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    // Flip the table to Enabled
+    env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
+      tableName,
+      TableState.State.ENABLED);
+    LOG.info("Table '" + tableName + "' was successfully enabled.");
+  }
+
+  /**
+   * Action after enabling table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void postEnable(final MasterProcedureEnv env, final EnableTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+
+  /**
+   * @param regionsInMeta
+   * @return List of regions neither in transition nor assigned.
+   * @throws IOException
+   */
+  private static Map<HRegionInfo, ServerName> regionsToAssignWithServerName(
+      final MasterProcedureEnv env,
+      final List<Pair<HRegionInfo, ServerName>> regionsInMeta) throws IOException {
+    Map<HRegionInfo, ServerName> regionsToAssign =
+        new HashMap<HRegionInfo, ServerName>(regionsInMeta.size());
+    RegionStates regionStates = env.getMasterServices().getAssignmentManager().getRegionStates();
+    for (Pair<HRegionInfo, ServerName> regionLocation : regionsInMeta) {
+      HRegionInfo hri = regionLocation.getFirst();
+      ServerName sn = regionLocation.getSecond();
+      if (regionStates.isRegionOffline(hri)) {
+        regionsToAssign.put(hri, sn);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skipping assign for the region " + hri + " during enable table "
+              + hri.getTable() + " because its already in tranition or assigned.");
+        }
+      }
+    }
+    return regionsToAssign;
+  }
+
+  /**
+   * Coprocessor Action.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void runCoprocessorAction(final MasterProcedureEnv env, final EnableTableState state)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          switch (state) {
+          case ENABLE_TABLE_PRE_OPERATION:
+            cpHost.preEnableTableHandler(getTableName());
+            break;
+          case ENABLE_TABLE_POST_OPERATION:
+            cpHost.postEnableTableHandler(getTableName());
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/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 76ca094..6928d02 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
@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.TableName;
 
 /**
  * Procedures that operates on a specific Table (e.g. create, delete, snapshot, ...)
@@ -29,7 +29,9 @@ import org.apache.hadoop.hbase.TableName;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public interface TableProcedureInterface {
-  public enum TableOperationType { CREATE, DELETE, EDIT, READ };
+  public enum TableOperationType {
+    CREATE, DELETE, DISABLE, EDIT, ENABLE, READ,
+  };
 
   /**
    * @return the name of the table the procedure is operating on

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
index bc97bb9..9bb436e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -36,7 +36,9 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@@ -172,6 +174,18 @@ public class MasterProcedureTestingUtility {
     return actualRegCount.get();
   }
 
+  public static void validateTableIsEnabled(final HMaster master, final TableName tableName)
+      throws IOException {
+    TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
+    assertTrue(tsm.getTableState(tableName).equals(TableState.State.ENABLED));
+  }
+
+  public static void validateTableIsDisabled(final HMaster master, final TableName tableName)
+      throws IOException {
+    TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
+    assertTrue(tsm.getTableState(tableName).equals(TableState.State.DISABLED));
+  }
+
   public static <TState> void testRecoveryAndDoubleExecution(
       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
       final int numSteps, final TState[] states) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
new file mode 100644
index 0000000..0537ccc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDisableTableProcedure.java
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotEnabledException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestDisableTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestDisableTableProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testDisableTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testDisableTable");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+
+    // Disable the table
+    long procId = procExec.submitProcedure(
+      new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  @Test(timeout = 60000)
+  public void testDisableTableMultipleTimes() throws Exception {
+    final TableName tableName = TableName.valueOf("testDisableTableMultipleTimes");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+
+    // Disable the table
+    long procId1 = procExec.submitProcedure(new DisableTableProcedure(
+        procExec.getEnvironment(), tableName, false));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+
+    // Disable the table again - expect failure
+    long procId2 = procExec.submitProcedure(new DisableTableProcedure(
+        procExec.getEnvironment(), tableName, false));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureResult result = procExec.getResult(procId2);
+    assertTrue(result.isFailed());
+    LOG.debug("Disable failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof TableNotEnabledException);
+
+    // Disable the table - expect failure from ProcedurePrepareLatch
+    try {
+      final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
+
+      long procId3 = procExec.submitProcedure(new DisableTableProcedure(
+          procExec.getEnvironment(), tableName, false, prepareLatch));
+      prepareLatch.await();
+      Assert.fail("Disable should throw exception through latch.");
+    } catch (TableNotEnabledException tnee) {
+      // Expected
+      LOG.debug("Disable failed with expected exception.");
+    }
+
+    // Disable the table again with skipping table state check flag (simulate recovery scenario)
+    long procId4 = procExec.submitProcedure(new DisableTableProcedure(
+        procExec.getEnvironment(), tableName, true));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId4);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId4);
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Disable procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new DisableTableProcedure(procExec.getEnvironment(), tableName,
+            false));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = DisableTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      DisableTableState.values());
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
new file mode 100644
index 0000000..12c78e8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestEnableTableProcedure.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestEnableTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestEnableTableProcedure.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static void setupConf(Configuration conf) {
+    conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
+  }
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
+    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testEnableTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testEnableTable");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // Enable the table
+    long procId = procExec.submitProcedure(
+      new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  @Test(timeout=60000, expected=TableNotDisabledException.class)
+  public void testEnableNonDisabledTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testEnableNonExistingTable");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+
+    // Enable the table - expect failure
+    long procId1 = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+
+    ProcedureResult result = procExec.getResult(procId1);
+    assertTrue(result.isFailed());
+    LOG.debug("Enable failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
+
+    // Enable the table with skipping table state check flag (simulate recovery scenario)
+    long procId2 = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, true));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+
+    // Enable the table - expect failure from ProcedurePrepareLatch
+    final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
+    long procId3 = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch));
+    prepareLatch.await();
+    Assert.fail("Enable should throw exception through latch.");
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Enable procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = EnableTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      EnableTableState.values());
+    MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Enable procedure && kill the executor
+    long procId = procExec.submitProcedure(
+        new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+
+    int numberOfSteps = EnableTableState.values().length - 2; // failing in the middle of proc
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      EnableTableState.values());
+    MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
+      tableName);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fbf41dd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
index faf7845..0f6c910 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -37,8 +37,11 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 
@@ -212,6 +215,79 @@ public class TestMasterFailoverWithProcedures {
   }
 
   // ==========================================================================
+  //  Test Disable Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testDisableTableWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestDisableTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testDisableTableWithFailoverAtStep(
+      DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE.ordinal());
+  }
+
+  private void testDisableTableWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testDisableTableWithFailoverAtStep" + step);
+
+    // create the table
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
+
+    MasterProcedureTestingUtility.validateTableIsDisabled(
+      UTIL.getHBaseCluster().getMaster(), tableName);
+  }
+
+  // ==========================================================================
+  //  Test Enable Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testEnableTableWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestEnableTableProcedure
+    // but without the master restart, only the executor/store is restarted.
+    // Without Master restart we may not find bug in the procedure code
+    // like missing "wait" for resources to be available (e.g. RS)
+    testEnableTableWithFailoverAtStep(
+      EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE.ordinal());
+  }
+
+  private void testEnableTableWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testEnableTableWithFailoverAtStep" + step);
+
+    // create the table
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
+
+    MasterProcedureTestingUtility.validateTableIsEnabled(
+      UTIL.getHBaseCluster().getMaster(), tableName);
+  }
+
+  // ==========================================================================
   //  Test Helpers
   // ==========================================================================
   public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,