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/10 00:59:53 UTC

[01/11] hbase git commit: HBASE-13210 Procedure V2 - master Modify table (Stephen Yuan Jiang)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 879f805d7 -> 9d919c916 (forced update)


http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/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 4e9b05e..a1ae971 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -58,6 +58,24 @@ message CreateTableStateData {
   repeated RegionInfo region_info = 3;
 }
 
+enum ModifyTableState {
+  MODIFY_TABLE_PREPARE = 1;
+  MODIFY_TABLE_PRE_OPERATION = 2;
+  MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;
+  MODIFY_TABLE_DELETE_FS_LAYOUT = 5;
+  MODIFY_TABLE_POST_OPERATION = 6;
+  MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;
+}
+
+message ModifyTableStateData {
+  required UserInformation user_info = 1;
+  optional TableSchema unmodified_table_schema = 2;
+  required TableSchema modified_table_schema = 3;
+  required bool delete_column_family_in_modify = 4;
+  repeated RegionInfo region_info = 5;
+}
+
 enum DeleteTableState {
   DELETE_TABLE_PRE_OPERATION = 1;
   DELETE_TABLE_REMOVE_FROM_META = 2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/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 e2e600c..ba739b2 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
@@ -18,10 +18,6 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -43,10 +39,11 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Maps;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Service;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -93,7 +90,6 @@ 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.ModifyTableHandler;
 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
@@ -102,15 +98,17 @@ import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -146,6 +144,11 @@ import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.servlet.Context;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Service;
+
 /**
  * HMaster is the "master server" for HBase. An HBase cluster has one active
  * master.  If many masters are started, all compete.  Whichever wins goes on to
@@ -1728,8 +1731,15 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     if (cpHost != null) {
       cpHost.preModifyTable(tableName, descriptor);
     }
+
     LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
-    new ModifyTableHandler(tableName, descriptor, this, this).prepare().process();
+
+    // Execute the operation synchronously - wait for the operation completes before continuing.
+    long procId = this.procedureExecutor.submitProcedure(
+        new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor));
+
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+
     if (cpHost != null) {
       cpHost.postModifyTable(tableName, descriptor);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index 78e4c11..de28cdc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -583,10 +583,12 @@ public class MasterFileSystem {
     Path familyDir = new Path(tableDir,
       new Path(region.getEncodedName(), Bytes.toString(familyName)));
     if (fs.delete(familyDir, true) == false) {
-      throw new IOException("Could not delete family "
-          + Bytes.toString(familyName) + " from FileSystem for region "
-          + region.getRegionNameAsString() + "(" + region.getEncodedName()
-          + ")");
+      if (fs.exists(familyDir)) {
+        throw new IOException("Could not delete family "
+            + Bytes.toString(familyName) + " from FileSystem for region "
+            + region.getRegionNameAsString() + "(" + region.getEncodedName()
+            + ")");
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
new file mode 100644
index 0000000..c6ff1b6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
@@ -0,0 +1,167 @@
+/**
+ * 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.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+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.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.BulkReOpen;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Helper class for schema change procedures
+ */
+@InterfaceAudience.Private
+public final class MasterDDLOperationHelper {
+  private static final Log LOG = LogFactory.getLog(MasterDDLOperationHelper.class);
+
+  private MasterDDLOperationHelper() {}
+
+  /**
+   * Check whether online schema change is allowed from config
+   **/
+  public static boolean isOnlineSchemaChangeAllowed(final MasterProcedureEnv env) {
+    return env.getMasterServices().getConfiguration()
+        .getBoolean("hbase.online.schema.update.enable", false);
+  }
+
+  /**
+   * Check whether a table is modifiable - exists and either offline or online with config set
+   * @param env MasterProcedureEnv
+   * @param tableName name of the table
+   * @throws IOException
+   */
+  public static void checkTableModifiable(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
+    // Checks whether the table exists
+    if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
+      throw new TableNotFoundException(tableName);
+    }
+
+    // We only execute this procedure with table online if online schema change config is set.
+    if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(tableName, TableState.State.DISABLED)
+        && !MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
+      throw new TableNotDisabledException(tableName);
+    }
+  }
+
+  /**
+   * Remove the column family from the file system
+   **/
+  public static void deleteColumnFamilyFromFileSystem(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      List<HRegionInfo> regionInfoList,
+      final byte[] familyName) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing family=" + Bytes.toString(familyName) + " from table=" + tableName);
+    }
+    if (regionInfoList == null) {
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, tableName);
+    }
+    for (HRegionInfo hri : regionInfoList) {
+      // Delete the family directory in FS for all the regions one by one
+      mfs.deleteFamilyFromFS(hri, familyName);
+    }
+  }
+
+  /**
+   * Reopen all regions from a table after a schema change operation.
+   **/
+  public static boolean reOpenAllRegions(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final List<HRegionInfo> regionInfoList) throws IOException {
+    boolean done = false;
+    LOG.info("Bucketing regions by region server...");
+    List<HRegionLocation> regionLocations = null;
+    Connection connection = env.getMasterServices().getConnection();
+    try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+      regionLocations = locator.getAllRegionLocations();
+    }
+    // Convert List<HRegionLocation> to Map<HRegionInfo, ServerName>.
+    NavigableMap<HRegionInfo, ServerName> hri2Sn = new TreeMap<HRegionInfo, ServerName>();
+    for (HRegionLocation location : regionLocations) {
+      hri2Sn.put(location.getRegionInfo(), location.getServerName());
+    }
+    TreeMap<ServerName, List<HRegionInfo>> serverToRegions = Maps.newTreeMap();
+    List<HRegionInfo> reRegions = new ArrayList<HRegionInfo>();
+    for (HRegionInfo hri : regionInfoList) {
+      ServerName sn = hri2Sn.get(hri);
+      // Skip the offlined split parent region
+      // See HBASE-4578 for more information.
+      if (null == sn) {
+        LOG.info("Skip " + hri);
+        continue;
+      }
+      if (!serverToRegions.containsKey(sn)) {
+        LinkedList<HRegionInfo> hriList = Lists.newLinkedList();
+        serverToRegions.put(sn, hriList);
+      }
+      reRegions.add(hri);
+      serverToRegions.get(sn).add(hri);
+    }
+
+    LOG.info("Reopening " + reRegions.size() + " regions on " + serverToRegions.size()
+        + " region servers.");
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    am.setRegionsToReopen(reRegions);
+    BulkReOpen bulkReopen = new BulkReOpen(env.getMasterServices(), serverToRegions, am);
+    while (true) {
+      try {
+        if (bulkReopen.bulkReOpen()) {
+          done = true;
+          break;
+        } else {
+          LOG.warn("Timeout before reopening all regions");
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("Reopen was interrupted");
+        // Preserve the interrupt.
+        Thread.currentThread().interrupt();
+        break;
+      }
+    }
+    return done;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
new file mode 100644
index 0000000..9dedde5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -0,0 +1,522 @@
+/**
+ * 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.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+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.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+public class ModifyTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, ModifyTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(ModifyTableProcedure.class);
+
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+  private HTableDescriptor unmodifiedHTableDescriptor = null;
+  private HTableDescriptor modifiedHTableDescriptor;
+  private UserGroupInformation user;
+  private List<HRegionInfo> regionInfoList;
+  private boolean deleteColumnFamilyInModify;
+
+  private Boolean traceEnabled = null;
+
+  public ModifyTableProcedure() {
+    initilize();
+  }
+
+  public ModifyTableProcedure(
+    final MasterProcedureEnv env,
+    final HTableDescriptor htd) throws IOException {
+    initilize();
+    this.modifiedHTableDescriptor = htd;
+    this.user = env.getRequestUser().getUGI();
+  }
+
+  private void initilize() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+    this.deleteColumnFamilyInModify = false;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final ModifyTableState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case MODIFY_TABLE_PREPARE:
+        prepareModify(env);
+        setNextState(ModifyTableState.MODIFY_TABLE_PRE_OPERATION);
+        break;
+      case MODIFY_TABLE_PRE_OPERATION:
+        preModify(env, state);
+        setNextState(ModifyTableState.MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(ModifyTableState.MODIFY_TABLE_REMOVE_REPLICA_COLUMN);
+        break;
+      case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
+        updateReplicaColumnsIfNeeded(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
+        if (deleteColumnFamilyInModify) {
+          setNextState(ModifyTableState.MODIFY_TABLE_DELETE_FS_LAYOUT);
+        } else {
+          setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
+        }
+        break;
+      case MODIFY_TABLE_DELETE_FS_LAYOUT:
+        deleteFromFs(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
+        setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
+        break;
+      case MODIFY_TABLE_POST_OPERATION:
+        postModify(env, state);
+        setNextState(ModifyTableState.MODIFY_TABLE_REOPEN_ALL_REGIONS);
+        break;
+      case MODIFY_TABLE_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn("Error trying to modify table=" + getTableName() + " state=" + state, e);
+      } else {
+        LOG.error("Error trying to modify table=" + getTableName() + " state=" + state, e);
+        setFailure("master-modify-table", e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final ModifyTableState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case MODIFY_TABLE_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case MODIFY_TABLE_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to un-modify?
+        break;
+      case MODIFY_TABLE_DELETE_FS_LAYOUT:
+        // Once we reach to this state - we could NOT rollback - as it is tricky to undelete
+        // the deleted files. We are not suppose to reach here, throw exception so that we know
+        // there is a code bug to investigate.
+        assert deleteColumnFamilyInModify;
+        throw new UnsupportedOperationException(this + " rollback of state=" + state
+            + " is unsupported.");
+      case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
+        // Undo the replica column update.
+        updateReplicaColumnsIfNeeded(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
+        break;
+      case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case MODIFY_TABLE_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to un-modify?
+        break;
+      case MODIFY_TABLE_PREPARE:
+        break; // Nothing to undo.
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      LOG.warn("Fail trying to rollback modify table=" + getTableName() + " state=" + state, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected ModifyTableState getState(final int stateId) {
+    return ModifyTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final ModifyTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyTableState getInitialState() {
+    return ModifyTableState.MODIFY_TABLE_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(final ModifyTableState state) {
+    if (aborted.get() && isRollbackSupported(state)) {
+      setAbortFailure("modify-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(
+      getTableName(),
+      EventType.C_M_MODIFY_TABLE.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(getTableName());
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
+        MasterProcedureProtos.ModifyTableStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setModifiedTableSchema(modifiedHTableDescriptor.convert())
+            .setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
+
+    if (unmodifiedHTableDescriptor != null) {
+      modifyTableMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+
+    if (regionInfoList != null) {
+      for (HRegionInfo hri : regionInfoList) {
+        modifyTableMsg.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+
+    modifyTableMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
+        MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo());
+    modifiedHTableDescriptor = HTableDescriptor.convert(modifyTableMsg.getModifiedTableSchema());
+    deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
+
+    if (modifyTableMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor =
+          HTableDescriptor.convert(modifyTableMsg.getUnmodifiedTableSchema());
+    }
+
+    if (modifyTableMsg.getRegionInfoCount() == 0) {
+      regionInfoList = null;
+    } else {
+      regionInfoList = new ArrayList<HRegionInfo>(modifyTableMsg.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri : modifyTableMsg.getRegionInfoList()) {
+        regionInfoList.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(getTableName());
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return modifiedHTableDescriptor.getTableName();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  /**
+   * Check conditions before any real action of modifying a table.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareModify(final MasterProcedureEnv env) throws IOException {
+    // Checks whether the table exists
+    if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), getTableName())) {
+      throw new TableNotFoundException(getTableName());
+    }
+
+    // In order to update the descriptor, we need to retrieve the old descriptor for comparison.
+    this.unmodifiedHTableDescriptor =
+        env.getMasterServices().getTableDescriptors().get(getTableName());
+
+    if (env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(getTableName(), TableState.State.ENABLED)) {
+      // We only execute this procedure with table online if online schema change config is set.
+      if (!MasterDDLOperationHelper.isOnlineSchemaChangeAllowed(env)) {
+        throw new TableNotDisabledException(getTableName());
+      }
+
+      if (modifiedHTableDescriptor.getRegionReplication() != unmodifiedHTableDescriptor
+          .getRegionReplication()) {
+        throw new IOException("REGION_REPLICATION change is not supported for enabled tables");
+      }
+    }
+
+    // Get the region info list before the real action.
+    this.regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+
+    // Find out whether all column families in unmodifiedHTableDescriptor also exists in
+    // the modifiedHTableDescriptor. This is to determine whether we are safe to rollback.
+    final Set<byte[]> oldFamilies = unmodifiedHTableDescriptor.getFamiliesKeys();
+    final Set<byte[]> newFamilies = modifiedHTableDescriptor.getFamiliesKeys();
+    for (byte[] familyName : oldFamilies) {
+      if (!newFamilies.contains(familyName)) {
+        this.deleteColumnFamilyInModify = true;
+        break;
+      }
+    }
+  }
+
+  /**
+   * Action before modifying table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preModify(final MasterProcedureEnv env, final ModifyTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Update descriptor
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   **/
+  private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    env.getMasterServices().getTableDescriptors().add(modifiedHTableDescriptor);
+  }
+
+  /**
+   * Undo the descriptor change (for rollback)
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   **/
+  private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
+
+    // delete any new column families from the modifiedHTableDescriptor.
+    deleteFromFs(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
+
+    // Make sure regions are opened after table descriptor is updated.
+    reOpenAllRegionsIfTableIsOnline(env);
+  }
+
+  /**
+   * Removes from hdfs the families that are not longer present in the new table descriptor.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void deleteFromFs(final MasterProcedureEnv env,
+      final HTableDescriptor oldHTableDescriptor, final HTableDescriptor newHTableDescriptor)
+      throws IOException {
+    final Set<byte[]> oldFamilies = oldHTableDescriptor.getFamiliesKeys();
+    final Set<byte[]> newFamilies = newHTableDescriptor.getFamiliesKeys();
+    for (byte[] familyName : oldFamilies) {
+      if (!newFamilies.contains(familyName)) {
+        MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(
+          env,
+          getTableName(),
+          regionInfoList,
+          familyName);
+      }
+    }
+  }
+
+  /**
+   * update replica column families if necessary.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void updateReplicaColumnsIfNeeded(
+    final MasterProcedureEnv env,
+    final HTableDescriptor oldHTableDescriptor,
+    final HTableDescriptor newHTableDescriptor) throws IOException {
+    final int oldReplicaCount = oldHTableDescriptor.getRegionReplication();
+    final int newReplicaCount = newHTableDescriptor.getRegionReplication();
+
+    if (newReplicaCount < oldReplicaCount) {
+      Set<byte[]> tableRows = new HashSet<byte[]>();
+      Connection connection = env.getMasterServices().getConnection();
+      Scan scan = MetaTableAccessor.getScanForTableName(connection, getTableName());
+      scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+
+      try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
+        ResultScanner resScanner = metaTable.getScanner(scan);
+        for (Result result : resScanner) {
+          tableRows.add(result.getRow());
+        }
+        MetaTableAccessor.removeRegionReplicasFromMeta(
+          tableRows,
+          newReplicaCount,
+          oldReplicaCount - newReplicaCount,
+          connection);
+      }
+    }
+
+    // Setup replication for region replicas if needed
+    if (newReplicaCount > 1 && oldReplicaCount <= 1) {
+      ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
+    }
+  }
+
+  /**
+   * Action after modifying table.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void postModify(final MasterProcedureEnv env, final ModifyTableState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Last action from the procedure - executed when online schema change is supported.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
+    // This operation only run when the table is enabled.
+    if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(getTableName(), TableState.State.ENABLED)) {
+      return;
+    }
+
+    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
+      LOG.info("Completed modify table operation on table " + getTableName());
+    } else {
+      LOG.warn("Error on reopening the regions on table " + getTableName());
+    }
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled whether the trace is enabled
+   */
+  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 ModifyTableState 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 MODIFY_TABLE_PRE_OPERATION:
+            cpHost.preModifyTableHandler(getTableName(), modifiedHTableDescriptor);
+            break;
+          case MODIFY_TABLE_POST_OPERATION:
+            cpHost.postModifyTableHandler(getTableName(), modifiedHTableDescriptor);
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+
+  /*
+   * Check whether we are in the state that can be rollback
+   */
+  private boolean isRollbackSupported(final ModifyTableState state) {
+    if (deleteColumnFamilyInModify) {
+      switch (state) {
+      case MODIFY_TABLE_DELETE_FS_LAYOUT:
+      case MODIFY_TABLE_POST_OPERATION:
+      case MODIFY_TABLE_REOPEN_ALL_REGIONS:
+        // It is not safe to rollback if we reach to these states.
+        return false;
+      default:
+        break;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
new file mode 100644
index 0000000..af29338
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -0,0 +1,403 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+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.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestModifyTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestModifyTableProcedure.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 testModifyTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testModifyTable");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // Modify the table descriptor
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+
+    // Test 1: Modify 1 property
+    long newMaxFileSize = htd.getMaxFileSize() * 2;
+    htd.setMaxFileSize(newMaxFileSize);
+    htd.setRegionReplication(3);
+
+    long procId1 = ProcedureTestingUtility.submitAndWait(
+        procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
+
+    HTableDescriptor currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
+
+    // Test 2: Modify multiple properties
+    boolean newReadOnlyOption = htd.isReadOnly() ? false : true;
+    long newMemStoreFlushSize = htd.getMemStoreFlushSize() * 2;
+    htd.setReadOnly(newReadOnlyOption);
+    htd.setMemStoreFlushSize(newMemStoreFlushSize);
+
+    long procId2 = ProcedureTestingUtility.submitAndWait(
+        procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
+
+    currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
+    assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
+  }
+
+  @Test(timeout = 60000)
+  public void testModifyTableAddCF() throws Exception {
+    final TableName tableName = TableName.valueOf("testModifyTableAddCF");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
+    HTableDescriptor currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(1, currentHtd.getFamiliesKeys().size());
+
+    // Test 1: Modify the table descriptor online
+    String cf2 = "cf2";
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    htd.addFamily(new HColumnDescriptor(cf2));
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+        procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+
+    currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(2, currentHtd.getFamiliesKeys().size());
+    assertTrue(currentHtd.hasFamily(cf2.getBytes()));
+
+    // Test 2: Modify the table descriptor offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    String cf3 = "cf3";
+    HTableDescriptor htd2 =
+        new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    htd2.addFamily(new HColumnDescriptor(cf3));
+
+    long procId2 =
+        ProcedureTestingUtility.submitAndWait(procExec,
+          new ModifyTableProcedure(procExec.getEnvironment(), htd2));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
+
+    currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertTrue(currentHtd.hasFamily(cf3.getBytes()));
+    assertEquals(3, currentHtd.getFamiliesKeys().size());
+  }
+
+  @Test(timeout = 60000)
+  public void testModifyTableDeleteCF() throws Exception {
+    final TableName tableName = TableName.valueOf("testModifyTableAddCF");
+    final String cf2 = "cf2";
+    final String cf3 = "cf3";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1", cf2, cf3);
+    HTableDescriptor currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(3, currentHtd.getFamiliesKeys().size());
+
+    // Test 1: Modify the table descriptor
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    htd.removeFamily(cf2.getBytes());
+
+    long procId = ProcedureTestingUtility.submitAndWait(
+        procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+
+    currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(2, currentHtd.getFamiliesKeys().size());
+    assertFalse(currentHtd.hasFamily(cf2.getBytes()));
+
+    // Test 2: Modify the table descriptor offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+
+    HTableDescriptor htd2 =
+        new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    htd2.removeFamily(cf3.getBytes());
+
+    long procId2 =
+        ProcedureTestingUtility.submitAndWait(procExec,
+          new ModifyTableProcedure(procExec.getEnvironment(), htd2));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
+
+    currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(1, currentHtd.getFamiliesKeys().size());
+    assertFalse(currentHtd.hasFamily(cf3.getBytes()));
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionOffline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
+    final String cf2 = "cf2";
+    final String cf3 = "cf3";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "cf1", cf3);
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Modify multiple properties of the table.
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
+    htd.setCompactionEnabled(newCompactionEnableOption);
+    htd.addFamily(new HColumnDescriptor(cf2));
+    htd.removeFamily(cf3.getBytes());
+    htd.setRegionReplication(3);
+
+    // Start the Modify procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = ModifyTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyTableState.values());
+
+    // Validate descriptor
+    HTableDescriptor currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
+    assertEquals(2, currentHtd.getFamiliesKeys().size());
+
+    // cf2 should be added cf3 should be removed
+    MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, false, "cf1", cf2);
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecutionOnline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
+    final String cf2 = "cf2";
+    final String cf3 = "cf3";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "cf1", cf3);
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Modify multiple properties of the table.
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
+    htd.setCompactionEnabled(newCompactionEnableOption);
+    htd.addFamily(new HColumnDescriptor(cf2));
+    htd.removeFamily(cf3.getBytes());
+
+    // Start the Modify procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = ModifyTableState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      ModifyTableState.values());
+
+    // Validate descriptor
+    HTableDescriptor currentHtd = UTIL.getHBaseAdmin().getTableDescriptor(tableName);
+    assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
+    assertEquals(2, currentHtd.getFamiliesKeys().size());
+    assertTrue(currentHtd.hasFamily(cf2.getBytes()));
+    assertFalse(currentHtd.hasFamily(cf3.getBytes()));
+
+    // cf2 should be added cf3 should be removed
+    MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, "cf1", cf2);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecutionOnline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final String familyName = "cf2";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "cf1");
+
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
+    htd.setCompactionEnabled(newCompactionEnableOption);
+    htd.addFamily(new HColumnDescriptor(familyName));
+
+    // Start the Modify procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
+
+    // Restart the executor and rollback the step twice
+    int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyTableState.values());
+
+    // cf2 should not be present
+    MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, "cf1");
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecutionOffline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final String familyName = "cf2";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "cf1");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    boolean newCompactionEnableOption = htd.isCompactionEnabled() ? false : true;
+    htd.setCompactionEnabled(newCompactionEnableOption);
+    htd.addFamily(new HColumnDescriptor(familyName));
+    htd.setRegionReplication(3);
+
+    // Start the Modify procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
+
+    // Restart the executor and rollback the step twice
+    int numberOfSteps = ModifyTableState.values().length - 4; // failing in the middle of proc
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyTableState.values());
+
+    // cf2 should not be present
+    MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, "cf1");
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecutionAfterPONR() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecutionAfterPONR");
+    final String familyToAddName = "cf2";
+    final String familyToRemove = "cf1";
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, familyToRemove);
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    HTableDescriptor htd = new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
+    htd.setCompactionEnabled(!htd.isCompactionEnabled());
+    htd.addFamily(new HColumnDescriptor(familyToAddName));
+    htd.removeFamily(familyToRemove.getBytes());
+    htd.setRegionReplication(3);
+
+    // Start the Modify procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new ModifyTableProcedure(procExec.getEnvironment(), htd));
+
+    // Failing after MODIFY_TABLE_DELETE_FS_LAYOUT we should not trigger the rollback.
+    // NOTE: the 5 (number of MODIFY_TABLE_DELETE_FS_LAYOUT + 1 step) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    int numberOfSteps = 5;
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecutionAfterPONR(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyTableState.values());
+
+    // "cf2" should be added and "cf1" should be removed
+    MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
+      tableName, regions, false, familyToAddName);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}


[04/11] hbase git commit: HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
new file mode 100644
index 0000000..dcf1940
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedure.java
@@ -0,0 +1,302 @@
+/**
+ * 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.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+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.DeleteColumnFamilyState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestDeleteColumnFamilyProcedure {
+  private static final Log LOG = LogFactory.getLog(TestDeleteColumnFamilyProcedure.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 testDeleteColumnFamily() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteColumnFamily");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    final String cf1 = "cf1";
+    final String cf2 = "cf2";
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, "f3");
+
+    // Test 1: delete the column family that exists online
+    long procId1 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf1.getBytes()));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf1);
+
+    // Test 2: delete the column family that exists offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    long procId2 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf2.getBytes()));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+  }
+
+  @Test(timeout=60000)
+  public void testDeleteColumnFamilyTwice() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteColumnFamilyTwice");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    final String cf2 = "cf2";
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2);
+
+    // delete the column family that exists
+    long procId1 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf2.getBytes()));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    // First delete should succeed
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
+
+    // delete the column family that does not exist
+    long procId2 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf2.getBytes()));
+
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+
+    // Second delete should fail with InvalidFamilyOperationException
+    ProcedureResult result = procExec.getResult(procId2);
+    assertTrue(result.isFailed());
+    LOG.debug("Delete online failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+
+    // Try again, this time with table disabled.
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    long procId3 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf2.getBytes()));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId3);
+    // Expect fail with InvalidFamilyOperationException
+    result = procExec.getResult(procId2);
+    assertTrue(result.isFailed());
+    LOG.debug("Delete offline failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+  }
+
+  @Test(timeout=60000)
+  public void testDeleteNonExistingColumnFamily() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    final String cf3 = "cf3";
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+
+    // delete the column family that does not exist
+    long procId1 =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf3.getBytes()));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+
+    ProcedureResult result = procExec.getResult(procId1);
+    assertTrue(result.isFailed());
+    LOG.debug("Delete failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionOffline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
+    final String cf4 = "cf4";
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3", cf4);
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf4.getBytes()));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = DeleteColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      DeleteColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf4);
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecutionOnline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
+    final String cf5 = "cf5";
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3", cf5);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, cf5.getBytes()));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = DeleteColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      DeleteColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf5);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final String cf5 = "cf5";
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "f1", "f2", "f3", cf5);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
+
+    // Failing before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should trigger the rollback
+    // NOTE: the 1 (number before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT step) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    int numberOfSteps = 1;
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      DeleteColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecutionAfterPONR() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecutionAfterPONR");
+    final String cf5 = "cf5";
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "f1", "f2", "f3", cf5);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
+
+    // Failing after DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should not trigger the rollback.
+    // NOTE: the 4 (number of DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT + 1 step) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    int numberOfSteps = 4;
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecutionAfterPONR(
+      procExec,
+      procId,
+      numberOfSteps,
+      DeleteColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, cf5);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
new file mode 100644
index 0000000..d29ea25
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyColumnFamilyProcedure.java
@@ -0,0 +1,238 @@
+/**
+ * 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.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+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.ModifyColumnFamilyState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestModifyColumnFamilyProcedure {
+  private static final Log LOG = LogFactory.getLog(TestModifyColumnFamilyProcedure.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 testModifyColumnFamily() throws Exception {
+    final TableName tableName = TableName.valueOf("testModifyColumnFamily");
+    final String cf1 = "cf1";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf1);
+    int oldBlockSize = columnDescriptor.getBlocksize();
+    int newBlockSize = 3 * oldBlockSize;
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, "f2");
+
+    // Test 1: modify the column family online
+    columnDescriptor.setBlocksize(newBlockSize);
+    long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
+        procExec.getEnvironment(), tableName, columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
+        .getMaster(), tableName, cf1, columnDescriptor);
+
+    // Test 2: modify the column family offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    columnDescriptor.setBlocksize(newBlockSize * 2);
+    long procId2 =
+        procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+    MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
+        .getMaster(), tableName, cf1, columnDescriptor);
+  }
+
+  @Test(timeout=60000)
+  public void testModifyNonExistingColumnFamily() throws Exception {
+    final TableName tableName = TableName.valueOf("testModifyExistingColumnFamily");
+    final String cf2 = "cf2";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2);
+    int oldBlockSize = columnDescriptor.getBlocksize();
+    int newBlockSize = 2 * oldBlockSize;
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
+
+    // Modify the column family that does not exist
+    columnDescriptor.setBlocksize(newBlockSize);
+    long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
+        procExec.getEnvironment(), tableName, columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+
+    ProcedureResult result = procExec.getResult(procId1);
+    assertTrue(result.isFailed());
+    LOG.debug("Modify failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecutionOffline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
+    final String cf3 = "cf3";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf3);
+    int oldBlockSize = columnDescriptor.getBlocksize();
+    int newBlockSize = 4 * oldBlockSize;
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf3);
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Modify procedure && kill the executor
+    columnDescriptor.setBlocksize(newBlockSize);
+    long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
+        procExec.getEnvironment(), tableName, columnDescriptor));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = ModifyColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
+        .getMaster(), tableName, cf3, columnDescriptor);
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecutionOnline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
+    final String cf4 = "cf4";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf4);
+    int oldBlockSize = columnDescriptor.getBlocksize();
+    int newBlockSize = 4 * oldBlockSize;
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf4);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Modify procedure && kill the executor
+    columnDescriptor.setBlocksize(newBlockSize);
+    long procId =
+        procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
+            tableName, columnDescriptor));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = ModifyColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      ModifyColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
+        .getMaster(), tableName, cf4, columnDescriptor);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final String cf3 = "cf3";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf3);
+    int oldBlockSize = columnDescriptor.getBlocksize();
+    int newBlockSize = 4 * oldBlockSize;
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf3);
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Modify procedure && kill the executor
+    columnDescriptor.setBlocksize(newBlockSize);
+    long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
+        procExec.getEnvironment(), tableName, columnDescriptor));
+
+    // Failing in the middle of proc
+    int numberOfSteps = ModifyColumnFamilyState.values().length - 2;
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+      procExec,
+      procId,
+      numberOfSteps,
+      ModifyColumnFamilyState.values());
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}


[02/11] hbase git commit: HBASE-13210 Procedure V2 - master Modify table (Stephen Yuan Jiang)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/1aa4d9d5/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
index 6d1694a..4deb131 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -127,6 +127,133 @@ public final class MasterProcedureProtos {
   }
 
   /**
+   * Protobuf enum {@code ModifyTableState}
+   */
+  public enum ModifyTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>MODIFY_TABLE_PREPARE = 1;</code>
+     */
+    MODIFY_TABLE_PREPARE(0, 1),
+    /**
+     * <code>MODIFY_TABLE_PRE_OPERATION = 2;</code>
+     */
+    MODIFY_TABLE_PRE_OPERATION(1, 2),
+    /**
+     * <code>MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR(2, 3),
+    /**
+     * <code>MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;</code>
+     */
+    MODIFY_TABLE_REMOVE_REPLICA_COLUMN(3, 4),
+    /**
+     * <code>MODIFY_TABLE_DELETE_FS_LAYOUT = 5;</code>
+     */
+    MODIFY_TABLE_DELETE_FS_LAYOUT(4, 5),
+    /**
+     * <code>MODIFY_TABLE_POST_OPERATION = 6;</code>
+     */
+    MODIFY_TABLE_POST_OPERATION(5, 6),
+    /**
+     * <code>MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;</code>
+     */
+    MODIFY_TABLE_REOPEN_ALL_REGIONS(6, 7),
+    ;
+
+    /**
+     * <code>MODIFY_TABLE_PREPARE = 1;</code>
+     */
+    public static final int MODIFY_TABLE_PREPARE_VALUE = 1;
+    /**
+     * <code>MODIFY_TABLE_PRE_OPERATION = 2;</code>
+     */
+    public static final int MODIFY_TABLE_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    public static final int MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR_VALUE = 3;
+    /**
+     * <code>MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;</code>
+     */
+    public static final int MODIFY_TABLE_REMOVE_REPLICA_COLUMN_VALUE = 4;
+    /**
+     * <code>MODIFY_TABLE_DELETE_FS_LAYOUT = 5;</code>
+     */
+    public static final int MODIFY_TABLE_DELETE_FS_LAYOUT_VALUE = 5;
+    /**
+     * <code>MODIFY_TABLE_POST_OPERATION = 6;</code>
+     */
+    public static final int MODIFY_TABLE_POST_OPERATION_VALUE = 6;
+    /**
+     * <code>MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;</code>
+     */
+    public static final int MODIFY_TABLE_REOPEN_ALL_REGIONS_VALUE = 7;
+
+
+    public final int getNumber() { return value; }
+
+    public static ModifyTableState valueOf(int value) {
+      switch (value) {
+        case 1: return MODIFY_TABLE_PREPARE;
+        case 2: return MODIFY_TABLE_PRE_OPERATION;
+        case 3: return MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR;
+        case 4: return MODIFY_TABLE_REMOVE_REPLICA_COLUMN;
+        case 5: return MODIFY_TABLE_DELETE_FS_LAYOUT;
+        case 6: return MODIFY_TABLE_POST_OPERATION;
+        case 7: return MODIFY_TABLE_REOPEN_ALL_REGIONS;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<ModifyTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<ModifyTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<ModifyTableState>() {
+            public ModifyTableState findValueByNumber(int number) {
+              return ModifyTableState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(1);
+    }
+
+    private static final ModifyTableState[] VALUES = values();
+
+    public static ModifyTableState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private ModifyTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:ModifyTableState)
+  }
+
+  /**
    * Protobuf enum {@code DeleteTableState}
    */
   public enum DeleteTableState
@@ -219,7 +346,7 @@ public final class MasterProcedureProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(1);
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(2);
     }
 
     private static final DeleteTableState[] VALUES = values();
@@ -489,19 +616,1257 @@ public final class MasterProcedureProtos {
       return regionInfo_;
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public int getRegionInfoCount() {
+      return regionInfo_.size();
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+      return regionInfo_.get(index);
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index) {
+      return regionInfo_.get(index);
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      regionInfo_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableSchema()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableSchema().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getRegionInfoCount(); i++) {
+        if (!getRegionInfo(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        output.writeMessage(3, regionInfo_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, regionInfo_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableSchema() == other.hasTableSchema());
+      if (hasTableSchema()) {
+        result = result && getTableSchema()
+            .equals(other.getTableSchema());
+      }
+      result = result && getRegionInfoList()
+          .equals(other.getRegionInfoList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableSchema()) {
+        hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getTableSchema().hashCode();
+      }
+      if (getRegionInfoCount() > 0) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfoList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code CreateTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableSchemaFieldBuilder();
+          getRegionInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+        } else {
+          tableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableSchemaBuilder_ == null) {
+          result.tableSchema_ = tableSchema_;
+        } else {
+          result.tableSchema_ = tableSchemaBuilder_.build();
+        }
+        if (regionInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.regionInfo_ = regionInfo_;
+        } else {
+          result.regionInfo_ = regionInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableSchema()) {
+          mergeTableSchema(other.getTableSchema());
+        }
+        if (regionInfoBuilder_ == null) {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfo_.isEmpty()) {
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureRegionInfoIsMutable();
+              regionInfo_.addAll(other.regionInfo_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfoBuilder_.isEmpty()) {
+              regionInfoBuilder_.dispose();
+              regionInfoBuilder_ = null;
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              regionInfoBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getRegionInfoFieldBuilder() : null;
+            } else {
+              regionInfoBuilder_.addAllMessages(other.regionInfo_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableSchema()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableSchema().isInitialized()) {
+          
+          return false;
+        }
+        for (int i = 0; i < getRegionInfoCount(); i++) {
+          if (!getRegionInfo(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableSchema table_schema = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public boolean hasTableSchema() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
+        if (tableSchemaBuilder_ == null) {
+          return tableSchema_;
+        } else {
+          return tableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (tableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableSchema_ = value;
+          onChanged();
+        } else {
+          tableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public Builder setTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (tableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            tableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(tableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            tableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          tableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public Builder clearTableSchema() {
+        if (tableSchemaBuilder_ == null) {
+          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          tableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
+        if (tableSchemaBuilder_ != null) {
+          return tableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return tableSchema_;
+        }
+      }
+      /**
+       * <code>required .TableSchema table_schema = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getTableSchemaFieldBuilder() {
+        if (tableSchemaBuilder_ == null) {
+          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  tableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          tableSchema_ = null;
+        }
+        return tableSchemaBuilder_;
+      }
+
+      // repeated .RegionInfo region_info = 3;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+        java.util.Collections.emptyList();
+      private void ensureRegionInfoIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+        if (regionInfoBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        } else {
+          return regionInfoBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public int getRegionInfoCount() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.size();
+        } else {
+          return regionInfoBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);
+        } else {
+          return regionInfoBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder addRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder addRegionInfo(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder addAllRegionInfo(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> values) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          super.addAll(values, regionInfo_);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+          onChanged();
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public Builder removeRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.remove(index);
+          onChanged();
+        } else {
+          regionInfoBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+          int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);  } else {
+          return regionInfoBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+           getRegionInfoOrBuilderList() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+        return getRegionInfoFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder> 
+           getRegionInfoBuilderList() {
+        return getRegionInfoFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  regionInfo_,
+                  ((bitField0_ & 0x00000004) == 0x00000004),
+                  getParentForChildren(),
+                  isClean());
+          regionInfo_ = null;
+        }
+        return regionInfoBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:CreateTableStateData)
+    }
+
+    static {
+      defaultInstance = new CreateTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:CreateTableStateData)
+  }
+
+  public interface ModifyTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // optional .TableSchema unmodified_table_schema = 2;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    boolean hasUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder();
+
+    // required .TableSchema modified_table_schema = 3;
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    boolean hasModifiedTableSchema();
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema();
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder();
+
+    // required bool delete_column_family_in_modify = 4;
+    /**
+     * <code>required bool delete_column_family_in_modify = 4;</code>
+     */
+    boolean hasDeleteColumnFamilyInModify();
+    /**
+     * <code>required bool delete_column_family_in_modify = 4;</code>
+     */
+    boolean getDeleteColumnFamilyInModify();
+
+    // repeated .RegionInfo region_info = 5;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code ModifyTableStateData}
+   */
+  public static final class ModifyTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements ModifyTableStateDataOrBuilder {
+    // Use ModifyTableStateData.newBuilder() to construct.
+    private ModifyTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ModifyTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ModifyTableStateData defaultInstance;
+    public static ModifyTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ModifyTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ModifyTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = unmodifiedTableSchema_.toBuilder();
+              }
+              unmodifiedTableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(unmodifiedTableSchema_);
+                unmodifiedTableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = modifiedTableSchema_.toBuilder();
+              }
+              modifiedTableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(modifiedTableSchema_);
+                modifiedTableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              deleteColumnFamilyInModify_ = input.readBool();
+              break;
+            }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ModifyTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<ModifyTableStateData>() {
+      public ModifyTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ModifyTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ModifyTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // optional .TableSchema unmodified_table_schema = 2;
+    public static final int UNMODIFIED_TABLE_SCHEMA_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    public boolean hasUnmodifiedTableSchema() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+      return unmodifiedTableSchema_;
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+      return unmodifiedTableSchema_;
+    }
+
+    // required .TableSchema modified_table_schema = 3;
+    public static final int MODIFIED_TABLE_SCHEMA_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_;
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    public boolean hasModifiedTableSchema() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema() {
+      return modifiedTableSchema_;
+    }
+    /**
+     * <code>required .TableSchema modified_table_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder() {
+      return modifiedTableSchema_;
+    }
+
+    // required bool delete_column_family_in_modify = 4;
+    public static final int DELETE_COLUMN_FAMILY_IN_MODIFY_FIELD_NUMBER = 4;
+    private boolean deleteColumnFamilyInModify_;
+    /**
+     * <code>required bool delete_column_family_in_modify = 4;</code>
+     */
+    public boolean hasDeleteColumnFamilyInModify() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required bool delete_column_family_in_modify = 4;</code>
+     */
+    public boolean getDeleteColumnFamilyInModify() {
+      return deleteColumnFamilyInModify_;
+    }
+
+    // repeated .RegionInfo region_info = 5;
+    public static final int REGION_INFO_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
      */
     public int getRegionInfoCount() {
       return regionInfo_.size();
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>repeated .RegionInfo region_info = 5;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
       return regionInfo_.get(index);
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>repeated .RegionInfo region_info = 5;</code>
      */
     public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
         int index) {
@@ -510,7 +1875,9 @@ public final class MasterProcedureProtos {
 
     private void initFields() {
       userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
-      tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      deleteColumnFamilyInModify_ = false;
       regionInfo_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
@@ -522,7 +1889,11 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasTableSchema()) {
+      if (!hasModifiedTableSchema()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDeleteColumnFamilyInModify()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -530,7 +1901,13 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!getTableSchema().isInitialized()) {
+      if (hasUnmodifiedTableSchema()) {
+        if (!getUnmodifiedTableSchema().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (!getModifiedTableSchema().isInitialized()) {
         memoizedIsInitialized = 0;
         return false;
       }
@@ -551,10 +1928,16 @@ public final class MasterProcedureProtos {
         output.writeMessage(1, userInfo_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, tableSchema_);
+        output.writeMessage(2, unmodifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, modifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(4, deleteColumnFamilyInModify_);
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
-        output.writeMessage(3, regionInfo_.get(i));
+        output.writeMessage(5, regionInfo_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -571,11 +1954,19 @@ public final class MasterProcedureProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, tableSchema_);
+          .computeMessageSize(2, unmodifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, modifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, deleteColumnFamilyInModify_);
       }
       for (int i = 0; i < regionInfo_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, regionInfo_.get(i));
+          .computeMessageSize(5, regionInfo_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -594,10 +1985,10 @@ public final class MasterProcedureProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) obj;
 
       boolean result = true;
       result = result && (hasUserInfo() == other.hasUserInfo());
@@ -605,10 +1996,20 @@ public final class MasterProcedureProtos {
         result = result && getUserInfo()
             .equals(other.getUserInfo());
       }
-      result = result && (hasTableSchema() == other.hasTableSchema());
-      if (hasTableSchema()) {
-        result = result && getTableSchema()
-            .equals(other.getTableSchema());
+      result = result && (hasUnmodifiedTableSchema() == other.hasUnmodifiedTableSchema());
+      if (hasUnmodifiedTableSchema()) {
+        result = result && getUnmodifiedTableSchema()
+            .equals(other.getUnmodifiedTableSchema());
+      }
+      result = result && (hasModifiedTableSchema() == other.hasModifiedTableSchema());
+      if (hasModifiedTableSchema()) {
+        result = result && getModifiedTableSchema()
+            .equals(other.getModifiedTableSchema());
+      }
+      result = result && (hasDeleteColumnFamilyInModify() == other.hasDeleteColumnFamilyInModify());
+      if (hasDeleteColumnFamilyInModify()) {
+        result = result && (getDeleteColumnFamilyInModify()
+            == other.getDeleteColumnFamilyInModify());
       }
       result = result && getRegionInfoList()
           .equals(other.getRegionInfoList());
@@ -629,9 +2030,17 @@ public final class MasterProcedureProtos {
         hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
         hash = (53 * hash) + getUserInfo().hashCode();
       }
-      if (hasTableSchema()) {
-        hash = (37 * hash) + TABLE_SCHEMA_FIELD_NUMBER;
-        hash = (53 * hash) + getTableSchema().hashCode();
+      if (hasUnmodifiedTableSchema()) {
+        hash = (37 * hash) + UNMODIFIED_TABLE_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getUnmodifiedTableSchema().hashCode();
+      }
+      if (hasModifiedTableSchema()) {
+        hash = (37 * hash) + MODIFIED_TABLE_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getModifiedTableSchema().hashCode();
+      }
+      if (hasDeleteColumnFamilyInModify()) {
+        hash = (37 * hash) + DELETE_COLUMN_FAMILY_IN_MODIFY_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getDeleteColumnFamilyInModify());
       }
       if (getRegionInfoCount() > 0) {
         hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
@@ -642,53 +2051,53 @@ public final class MasterProcedureProtos {
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -697,7 +2106,7 @@ public final class MasterProcedureProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -709,24 +2118,24 @@ public final class MasterProcedureProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code CreateTableStateData}
+     * Protobuf type {@code ModifyTableStateData}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateDataOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateDataOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -739,7 +2148,8 @@ public final class MasterProcedureProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
-          getTableSchemaFieldBuilder();
+          getUnmodifiedTableSchemaFieldBuilder();
+          getModifiedTableSchemaFieldBuilder();
           getRegionInfoFieldBuilder();
         }
       }
@@ -755,15 +2165,23 @@ public final class MasterProcedureProtos {
           userInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        if (tableSchemaBuilder_ == null) {
-          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
         } else {
-          tableSchemaBuilder_.clear();
+          unmodifiedTableSchemaBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+        } else {
+          modifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        deleteColumnFamilyInModify_ = false;
+        bitField0_ = (bitField0_ & ~0x00000008);
         if (regionInfoBuilder_ == null) {
           regionInfo_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000004);
+          bitField0_ = (bitField0_ & ~0x00000010);
         } else {
           regionInfoBuilder_.clear();
         }
@@ -776,23 +2194,23 @@ public final class MasterProcedureProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_CreateTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -806,15 +2224,27 @@ public final class MasterProcedureProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        if (tableSchemaBuilder_ == null) {
-          result.tableSchema_ = tableSchema_;
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchema_;
         } else {
-          result.tableSchema_ = tableSchemaBuilder_.build();
+          result.unmodifiedTableSchema_ = unmodifiedTableSchemaBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (modifiedTableSchemaBuilder_ == null) {
+          result.modifiedTableSchema_ = modifiedTableSchema_;
+        } else {
+          result.modifiedTableSchema_ = modifiedTableSchemaBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
         }
+        result.deleteColumnFamilyInModify_ = deleteColumnFamilyInModify_;
         if (regionInfoBuilder_ == null) {
-          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
             regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
-            bitField0_ = (bitField0_ & ~0x00000004);
+            bitField0_ = (bitField0_ & ~0x00000010);
           }
           result.regionInfo_ = regionInfo_;
         } else {
@@ -826,27 +2256,33 @@ public final class MasterProcedureProtos {
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.getDefaultInstance()) return this;
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.getDefaultInstance()) return this;
         if (other.hasUserInfo()) {
           mergeUserInfo(other.getUserInfo());
         }
-        if (other.hasTableSchema()) {
-          mergeTableSchema(other.getTableSchema());
+        if (other.hasUnmodifiedTableSchema()) {
+          mergeUnmodifiedTableSchema(other.getUnmodifiedTableSchema());
+        }
+        if (other.hasModifiedTableSchema()) {
+          mergeModifiedTableSchema(other.getModifiedTableSchema());
+        }
+        if (other.hasDeleteColumnFamilyInModify()) {
+          setDeleteColumnFamilyInModify(other.getDeleteColumnFamilyInModify());
         }
         if (regionInfoBuilder_ == null) {
           if (!other.regionInfo_.isEmpty()) {
             if (regionInfo_.isEmpty()) {
               regionInfo_ = other.regionInfo_;
-              bitField0_ = (bitField0_ & ~0x00000004);
+              bitField0_ = (bitField0_ & ~0x00000010);
             } else {
               ensureRegionInfoIsMutable();
               regionInfo_.addAll(other.regionInfo_);
@@ -859,7 +2295,7 @@ public final class MasterProcedureProtos {
               regionInfoBuilder_.dispose();
               regionInfoBuilder_ = null;
               regionInfo_ = other.regionInfo_;
-              bitField0_ = (bitField0_ & ~0x00000004);
+              bitField0_ = (bitField0_ & ~0x00000010);
               regionInfoBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getRegionInfoFieldBuilder() : null;
@@ -877,7 +2313,11 @@ public final class MasterProcedureProtos {
           
           return false;
         }
-        if (!hasTableSchema()) {
+        if (!hasModifiedTableSchema()) {
+          
+          return false;
+        }
+        if (!hasDeleteColumnFamilyInModify()) {
           
           return false;
         }
@@ -885,7 +2325,13 @@ public final class MasterProcedureProtos {
           
           return false;
         }
-        if (!getTableSchema().isInitialized()) {
+        if (hasUnmodifiedTableSchema()) {
+          if (!getUnmodifiedTableSchema().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (!getModifiedTableSchema().isInitialized()) {
           
           return false;
         }
@@ -902,11 +2348,11 @@ public final class MasterProcedureProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) e.getUnfinishedMessage();
           throw e;
         } finally {
           if (parsedMessage != null) {
@@ -1034,130 +2480,280 @@ public final class MasterProcedureProtos {
         return userInfoBuilder_;
       }
 
-      // required .TableSchema table_schema = 2;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      // optional .TableSchema unmodified_table_schema = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> tableSchemaBuilder_;
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public boolean hasTableSchema() {
+      public boolean hasUnmodifiedTableSchema() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema() {
-        if (tableSchemaBuilder_ == null) {
-          return tableSchema_;
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          return unmodifiedTableSchema_;
         } else {
-          return tableSchemaBuilder_.getMessage();
+          return unmodifiedTableSchemaBuilder_.getMessage();
         }
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public Builder setTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
-        if (tableSchemaBuilder_ == null) {
+      public Builder setUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
           }
-          tableSchema_ = value;
+          unmodifiedTableSchema_ = value;
           onChanged();
         } else {
-          tableSchemaBuilder_.setMessage(value);
+          unmodifiedTableSchemaBuilder_.setMessage(value);
         }
         bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public Builder setTableSchema(
+      public Builder setUnmodifiedTableSchema(
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
-        if (tableSchemaBuilder_ == null) {
-          tableSchema_ = builderForValue.build();
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = builderForValue.build();
           onChanged();
         } else {
-          tableSchemaBuilder_.setMessage(builderForValue.build());
+          unmodifiedTableSchemaBuilder_.setMessage(builderForValue.build());
         }
         bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public Builder mergeTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
-        if (tableSchemaBuilder_ == null) {
+      public Builder mergeUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
           if (((bitField0_ & 0x00000002) == 0x00000002) &&
-              tableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
-            tableSchema_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(tableSchema_).mergeFrom(value).buildPartial();
+              unmodifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            unmodifiedTableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(unmodifiedTableSchema_).mergeFrom(value).buildPartial();
           } else {
-            tableSchema_ = value;
+            unmodifiedTableSchema_ = value;
           }
           onChanged();
         } else {
-          tableSchemaBuilder_.mergeFrom(value);
+          unmodifiedTableSchemaBuilder_.mergeFrom(value);
         }
         bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public Builder clearTableSchema() {
-        if (tableSchemaBuilder_ == null) {
-          tableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      public Builder clearUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
           onChanged();
         } else {
-          tableSchemaBuilder_.clear();
+          unmodifiedTableSchemaBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getTableSchemaBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
         bitField0_ |= 0x00000002;
         onChanged();
-        return getTableSchemaFieldBuilder().getBuilder();
+        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
-        if (tableSchemaBuilder_ != null) {
-          return tableSchemaBuilder_.getMessageOrBuilder();
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+        if (unmodifiedTableSchemaBuilder_ != null) {
+          return unmodifiedTableSchemaBuilder_.getMessageOrBuilder();
         } else {
-          return tableSchema_;
+          return unmodifiedTableSchema_;
         }
       }
       /**
-       * <code>required .TableSchema table_schema = 2;</code>
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
-          getTableSchemaFieldBuilder() {
-        if (tableSchemaBuilder_ == null) {
-          tableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+          getUnmodifiedTableSchemaFieldBuilder() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org

<TRUNCATED>

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

Posted by mb...@apache.org.
HBASE-13211 Procedure V2 - master Enable/Disable table (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: ec87cf1bba925cef47f8f9d0857dc9392d4b248f
Parents: 0be8fcc
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:52:02 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 23:58:32 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 2252 +++++++++++++++++-
 .../src/main/protobuf/MasterProcedure.proto     |   30 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   41 +-
 .../master/procedure/DisableTableProcedure.java |  542 +++++
 .../master/procedure/EnableTableProcedure.java  |  586 +++++
 .../procedure/TableProcedureInterface.java      |    6 +-
 .../MasterProcedureTestingUtility.java          |   14 +
 .../procedure/TestDisableTableProcedure.java    |  182 ++
 .../procedure/TestEnableTableProcedure.java     |  193 ++
 .../TestMasterFailoverWithProcedures.java       |   76 +
 10 files changed, 3801 insertions(+), 121 deletions(-)
----------------------------------------------------------------------



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

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ec87cf1b/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 ea8169e..9c0a369 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -139,3 +139,33 @@ message DeleteColumnFamilyStateData {
   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 EnableTableStateData {
+  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 DisableTableStateData {
+  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/ec87cf1b/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/ec87cf1b/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..2e5bdb3
--- /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.DisableTableStateData.Builder disableTableMsg =
+        MasterProcedureProtos.DisableTableStateData.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.DisableTableStateData disableTableMsg =
+        MasterProcedureProtos.DisableTableStateData.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/ec87cf1b/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..121a0a9
--- /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.EnableTableStateData.Builder enableTableMsg =
+        MasterProcedureProtos.EnableTableStateData.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.EnableTableStateData enableTableMsg =
+        MasterProcedureProtos.EnableTableStateData.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/ec87cf1b/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/ec87cf1b/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/ec87cf1b/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/ec87cf1b/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/ec87cf1b/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,


[05/11] hbase git commit: HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/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 a1ae971..ea8169e 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -90,3 +90,52 @@ message DeleteTableStateData {
   required TableName table_name = 2;
   repeated RegionInfo region_info = 3;
 }
+
+enum AddColumnFamilyState {
+  ADD_COLUMN_FAMILY_PREPARE = 1;
+  ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
+  ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  ADD_COLUMN_FAMILY_POST_OPERATION = 4;
+  ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message AddColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}
+
+enum ModifyColumnFamilyState {
+  MODIFY_COLUMN_FAMILY_PREPARE = 1;
+  MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
+  MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
+  MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message ModifyColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}
+
+enum DeleteColumnFamilyState {
+  DELETE_COLUMN_FAMILY_PREPARE = 1;
+  DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
+  DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
+  DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
+  DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
+}
+
+message DeleteColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bytes columnfamily_name = 3;
+  optional TableSchema unmodified_table_schema = 4;
+  repeated RegionInfo region_info = 5;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/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 ba739b2..2e33095 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
@@ -90,14 +90,14 @@ 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.TableAddFamilyHandler;
-import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
-import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
 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.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -1618,8 +1618,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
         return;
       }
     }
-    //TODO: we should process this (and some others) in an executor
-    new TableAddFamilyHandler(tableName, columnDescriptor, this, this).prepare().process();
+    // Execute the operation synchronously - wait for the operation to complete before continuing.
+    long procId =
+        this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor
+            .getEnvironment(), tableName, columnDescriptor));
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
     if (cpHost != null) {
       cpHost.postAddColumn(tableName, columnDescriptor);
     }
@@ -1637,8 +1640,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       }
     }
     LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
-    new TableModifyFamilyHandler(tableName, descriptor, this, this)
-      .prepare().process();
+
+    // Execute the operation synchronously - wait for the operation to complete before continuing.
+    long procId =
+        this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor
+            .getEnvironment(), tableName, descriptor));
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+
     if (cpHost != null) {
       cpHost.postModifyColumn(tableName, descriptor);
     }
@@ -1654,7 +1662,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       }
     }
     LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
-    new TableDeleteFamilyHandler(tableName, columnName, this, this).prepare().process();
+
+    // Execute the operation synchronously - wait for the operation to complete before continuing.
+    long procId =
+        this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor
+            .getEnvironment(), tableName, columnName));
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+
     if (cpHost != null) {
       cpHost.postDeleteColumn(tableName, columnName);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
index 7b5c5c5..3bbef0a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.master.handler;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Handles adding a new family to an existing table.
+ * Handles Deleting a column family from an existing table.
  */
 @InterfaceAudience.Private
 public class TableDeleteFamilyHandler extends TableEventHandler {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
new file mode 100644
index 0000000..27d17d2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
@@ -0,0 +1,416 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * The procedure to add a column family to an existing table.
+ */
+@InterfaceAudience.Private
+public class AddColumnFamilyProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, AddColumnFamilyState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
+
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+  private TableName tableName;
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private HColumnDescriptor cfDescriptor;
+  private UserGroupInformation user;
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled;
+
+  public AddColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  public AddColumnFamilyProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final HColumnDescriptor cfDescriptor) throws IOException {
+    this.tableName = tableName;
+    this.cfDescriptor = cfDescriptor;
+    this.user = env.getRequestUser().getUGI();
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final AddColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case ADD_COLUMN_FAMILY_PREPARE:
+        prepareAdd(env);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case ADD_COLUMN_FAMILY_PRE_OPERATION:
+        preAdd(env, state);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case ADD_COLUMN_FAMILY_POST_OPERATION:
+        postAdd(env, state);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.warn("Error trying to add the column family" + getColumnFamilyName() + " to the table "
+          + tableName + " (in state=" + state + ")", e);
+
+      setFailure("master-add-columnfamily", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final AddColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case ADD_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case ADD_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case ADD_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for adding the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected AddColumnFamilyState getState(final int stateId) {
+    return AddColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final AddColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected AddColumnFamilyState getInitialState() {
+    return AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(AddColumnFamilyState state) {
+    if (aborted.get()) {
+      setAbortFailure("add-columnfamily", "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_ADD_FAMILY.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.AddColumnFamilyStateData.Builder addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilySchema(cfDescriptor.convert());
+    if (unmodifiedHTableDescriptor != null) {
+      addCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+    if (regionInfoList != null) {
+      for (HRegionInfo hri : regionInfoList) {
+        addCFMsg.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+
+    addCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
+    cfDescriptor = HColumnDescriptor.convert(addCFMsg.getColumnfamilySchema());
+    if (addCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(addCFMsg.getUnmodifiedTableSchema());
+    }
+    if (addCFMsg.getRegionInfoCount() == 0) {
+      regionInfoList = null;
+    } else {
+      regionInfoList = new ArrayList<HRegionInfo>(addCFMsg.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri : addCFMsg.getRegionInfoList()) {
+        regionInfoList.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(", columnfamily=");
+    if (cfDescriptor != null) {
+      sb.append(getColumnFamilyName());
+    } else {
+      sb.append("Unknown");
+    }
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  /**
+   * Action before any real action of adding column family.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareAdd(final MasterProcedureEnv env) throws IOException {
+    // Checks whether the table is allowed to be modified.
+    MasterDDLOperationHelper.checkTableModifiable(env, tableName);
+
+    // In order to update the descriptor, we need to retrieve the old descriptor for comparison.
+    unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
+    if (unmodifiedHTableDescriptor == null) {
+      throw new IOException("HTableDescriptor missing for " + tableName);
+    }
+    if (unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
+      throw new InvalidFamilyOperationException("Column family '" + getColumnFamilyName()
+          + "' in table '" + tableName + "' already exists so cannot be added");
+    }
+
+    // Get the region info list before the real action.
+    this.regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+  }
+
+  /**
+   * Action before adding column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preAdd(final MasterProcedureEnv env, final AddColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Add the column family to the file system
+   */
+  private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    // Update table descriptor
+    LOG.info("AddColumn. Table = " + tableName + " HCD = " + cfDescriptor.toString());
+
+    HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
+
+    if (htd.hasFamily(cfDescriptor.getName())) {
+      // It is possible to reach this situation, as we could already add the column family
+      // to table descriptor, but the master failover happens before we complete this state.
+      // We should be able to handle running this function multiple times without causing problem.
+      return;
+    }
+
+    htd.addFamily(cfDescriptor);
+    env.getMasterServices().getTableDescriptors().add(htd);
+  }
+
+  /**
+   * Restore the table descriptor back to pre-add
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   **/
+  private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
+    if (htd.hasFamily(cfDescriptor.getName())) {
+      // Remove the column family from file system and update the table descriptor to
+      // the before-add-column-family-state
+      MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName, regionInfoList,
+        cfDescriptor.getName());
+
+      env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
+
+      // Make sure regions are opened after table descriptor is updated.
+      reOpenAllRegionsIfTableIsOnline(env);
+    }
+  }
+
+  /**
+   * Action after adding column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void postAdd(final MasterProcedureEnv env, final AddColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Last action from the procedure - executed when online schema change is supported.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
+    // This operation only run when the table is enabled.
+    if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(getTableName(), TableState.State.ENABLED)) {
+      return;
+    }
+
+    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
+      LOG.info("Completed add column family operation on table " + getTableName());
+    } else {
+      LOG.warn("Error on reopening the regions on table " + getTableName());
+    }
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+
+  private String getColumnFamilyName() {
+    return cfDescriptor.getNameAsString();
+  }
+
+  /**
+   * Coprocessor Action.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void runCoprocessorAction(final MasterProcedureEnv env, final AddColumnFamilyState 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 ADD_COLUMN_FAMILY_PRE_OPERATION:
+            cpHost.preAddColumnHandler(tableName, cfDescriptor);
+            break;
+          case ADD_COLUMN_FAMILY_POST_OPERATION:
+            cpHost.postAddColumnHandler(tableName, cfDescriptor);
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
new file mode 100644
index 0000000..979dcef
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -0,0 +1,448 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+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.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
+import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * The procedure to delete a column family from an existing table.
+ */
+@InterfaceAudience.Private
+public class DeleteColumnFamilyProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, DeleteColumnFamilyState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
+
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private TableName tableName;
+  private byte [] familyName;
+  private UserGroupInformation user;
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled;
+
+  public DeleteColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  public DeleteColumnFamilyProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final byte[] familyName) throws IOException {
+    this.tableName = tableName;
+    this.familyName = familyName;
+    this.user = env.getRequestUser().getUGI();
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, DeleteColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case DELETE_COLUMN_FAMILY_PREPARE:
+        prepareDelete(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case DELETE_COLUMN_FAMILY_PRE_OPERATION:
+        preDelete(env, state);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT);
+        break;
+      case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
+        deleteFromFs(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case DELETE_COLUMN_FAMILY_POST_OPERATION:
+        postDelete(env, state);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn("Error trying to delete the column family " + getColumnFamilyName()
+          + " from table " + tableName + "(in state=" + state + ")", e);
+      } else {
+        LOG.error("Error trying to delete the column family " + getColumnFamilyName()
+          + " from table " + tableName + "(in state=" + state + ")", e);
+        setFailure("master-delete-column-family", e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case DELETE_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
+        // Once we reach to this state - we could NOT rollback - as it is tricky to undelete
+        // the deleted files. We are not suppose to reach here, throw exception so that we know
+        // there is a code bug to investigate.
+        throw new UnsupportedOperationException(this + " rollback of state=" + state
+            + " is unsupported.");
+      case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case DELETE_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case DELETE_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for deleting the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected DeleteColumnFamilyState getState(final int stateId) {
+    return DeleteColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final DeleteColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected DeleteColumnFamilyState getInitialState() {
+    return DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(DeleteColumnFamilyState state) {
+    if (aborted.get() && isRollbackSupported(state)) {
+      setAbortFailure("delete-columnfamily", "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_DELETE_FAMILY.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.DeleteColumnFamilyStateData.Builder deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilyName(ByteStringer.wrap(familyName));
+    if (unmodifiedHTableDescriptor != null) {
+      deleteCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+    if (regionInfoList != null) {
+      for (HRegionInfo hri : regionInfoList) {
+        deleteCFMsg.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+
+    deleteCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+    MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
+    familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
+
+    if (deleteCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(deleteCFMsg.getUnmodifiedTableSchema());
+    }
+    if (deleteCFMsg.getRegionInfoCount() == 0) {
+      regionInfoList = null;
+    } else {
+      regionInfoList = new ArrayList<HRegionInfo>(deleteCFMsg.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri : deleteCFMsg.getRegionInfoList()) {
+        regionInfoList.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(", columnfamily=");
+    if (familyName != null) {
+      sb.append(getColumnFamilyName());
+    } else {
+      sb.append("Unknown");
+    }
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  /**
+   * Action before any real action of deleting column family.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareDelete(final MasterProcedureEnv env) throws IOException {
+    // Checks whether the table is allowed to be modified.
+    MasterDDLOperationHelper.checkTableModifiable(env, tableName);
+
+    // In order to update the descriptor, we need to retrieve the old descriptor for comparison.
+    unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
+    if (unmodifiedHTableDescriptor == null) {
+      throw new IOException("HTableDescriptor missing for " + tableName);
+    }
+    if (!unmodifiedHTableDescriptor.hasFamily(familyName)) {
+      throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+          + "' does not exist, so it cannot be deleted");
+    }
+
+    // Get the region info list before the real action.
+    this.regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+  }
+
+  /**
+   * Action before deleting column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preDelete(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Remove the column family from the file system and update the table descriptor
+   */
+  private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    // Update table descriptor
+    LOG.info("DeleteColumn. Table = " + tableName + " family = " + getColumnFamilyName());
+
+    HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
+
+    if (!htd.hasFamily(familyName)) {
+      // It is possible to reach this situation, as we could already delete the column family
+      // from table descriptor, but the master failover happens before we complete this state.
+      // We should be able to handle running this function multiple times without causing problem.
+      return;
+    }
+
+    htd.removeFamily(familyName);
+    env.getMasterServices().getTableDescriptors().add(htd);
+  }
+
+  /**
+   * Restore back to the old descriptor
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   **/
+  private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
+
+    // Make sure regions are opened after table descriptor is updated.
+    reOpenAllRegionsIfTableIsOnline(env);
+  }
+
+  /**
+   * Remove the column family from the file system
+   **/
+  private void deleteFromFs(final MasterProcedureEnv env) throws IOException {
+    MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName, regionInfoList,
+      familyName);
+  }
+
+  /**
+   * Action after deleting column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void postDelete(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Last action from the procedure - executed when online schema change is supported.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
+    // This operation only run when the table is enabled.
+    if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(getTableName(), TableState.State.ENABLED)) {
+      return;
+    }
+
+    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
+      LOG.info("Completed delete column family operation on table " + getTableName());
+    } else {
+      LOG.warn("Error on reopening the regions on table " + getTableName());
+    }
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+
+  private String getColumnFamilyName() {
+    return Bytes.toString(familyName);
+  }
+
+  /**
+   * Coprocessor Action.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void runCoprocessorAction(final MasterProcedureEnv env,
+      final DeleteColumnFamilyState 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 DELETE_COLUMN_FAMILY_PRE_OPERATION:
+            cpHost.preDeleteColumnHandler(tableName, familyName);
+            break;
+          case DELETE_COLUMN_FAMILY_POST_OPERATION:
+            cpHost.postDeleteColumnHandler(tableName, familyName);
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+
+  /*
+   * Check whether we are in the state that can be rollback
+   */
+  private boolean isRollbackSupported(final DeleteColumnFamilyState state) {
+    switch (state) {
+    case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+    case DELETE_COLUMN_FAMILY_POST_OPERATION:
+    case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
+        // It is not safe to rollback if we reach to these states.
+        return false;
+      default:
+        break;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
new file mode 100644
index 0000000..a3a804a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
@@ -0,0 +1,400 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * The procedure to modify a column family from an existing table.
+ */
+@InterfaceAudience.Private
+public class ModifyColumnFamilyProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, ModifyColumnFamilyState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
+
+  private AtomicBoolean aborted = new AtomicBoolean(false);
+  private TableName tableName;
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private HColumnDescriptor cfDescriptor;
+  private UserGroupInformation user;
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled;
+
+  public ModifyColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  public ModifyColumnFamilyProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final HColumnDescriptor cfDescriptor) throws IOException {
+    this.tableName = tableName;
+    this.cfDescriptor = cfDescriptor;
+    this.user = env.getRequestUser().getUGI();
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env,
+      final ModifyColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case MODIFY_COLUMN_FAMILY_PREPARE:
+        prepareModify(env);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
+        preModify(env, state);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case MODIFY_COLUMN_FAMILY_POST_OPERATION:
+        postModify(env, state);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.warn("Error trying to modify the column family " + getColumnFamilyName()
+          + " of the table " + tableName + "(in state=" + state + ")", e);
+
+      setFailure("master-modify-columnfamily", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case MODIFY_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case MODIFY_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for adding the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected ModifyColumnFamilyState getState(final int stateId) {
+    return ModifyColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final ModifyColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyColumnFamilyState getInitialState() {
+    return ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(ModifyColumnFamilyState state) {
+    if (aborted.get()) {
+      setAbortFailure("modify-columnfamily", "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_MODIFY_FAMILY.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.ModifyColumnFamilyStateData.Builder modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilySchema(cfDescriptor.convert());
+    if (unmodifiedHTableDescriptor != null) {
+      modifyCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+    if (regionInfoList != null) {
+      for (HRegionInfo hri : regionInfoList) {
+        modifyCFMsg.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+
+    modifyCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
+    cfDescriptor = HColumnDescriptor.convert(modifyCFMsg.getColumnfamilySchema());
+    if (modifyCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(modifyCFMsg.getUnmodifiedTableSchema());
+    }
+    if (modifyCFMsg.getRegionInfoCount() == 0) {
+      regionInfoList = null;
+    } else {
+      regionInfoList = new ArrayList<HRegionInfo>(modifyCFMsg.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri : modifyCFMsg.getRegionInfoList()) {
+        regionInfoList.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(tableName);
+    sb.append(", columnfamily=");
+    if (cfDescriptor != null) {
+      sb.append(getColumnFamilyName());
+    } else {
+      sb.append("Unknown");
+    }
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.EDIT;
+  }
+
+  /**
+   * Action before any real action of modifying column family.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void prepareModify(final MasterProcedureEnv env) throws IOException {
+    // Checks whether the table is allowed to be modified.
+    MasterDDLOperationHelper.checkTableModifiable(env, tableName);
+
+    unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
+    if (unmodifiedHTableDescriptor == null) {
+      throw new IOException("HTableDescriptor missing for " + tableName);
+    }
+    if (!unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
+      throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+          + "' does not exist, so it cannot be modified");
+    }
+
+    // Get the region info list before the real action.
+    this.regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+  }
+
+  /**
+   * Action before modifying column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void preModify(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Modify the column family from the file system
+   */
+  private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    // Update table descriptor
+    LOG.info("ModifyColumnFamily. Table = " + tableName + " HCD = " + cfDescriptor.toString());
+
+    HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
+    htd.modifyFamily(cfDescriptor);
+    env.getMasterServices().getTableDescriptors().add(htd);
+  }
+
+  /**
+   * Restore back to the old descriptor
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   **/
+  private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
+    env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
+
+    // Make sure regions are opened after table descriptor is updated.
+    reOpenAllRegionsIfTableIsOnline(env);
+  }
+
+  /**
+   * Action after modifying column family.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void postModify(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
+      throws IOException, InterruptedException {
+    runCoprocessorAction(env, state);
+  }
+
+  /**
+   * Last action from the procedure - executed when online schema change is supported.
+   * @param env MasterProcedureEnv
+   * @throws IOException
+   */
+  private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
+    // This operation only run when the table is enabled.
+    if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
+        .isTableState(getTableName(), TableState.State.ENABLED)) {
+      return;
+    }
+
+    if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
+      LOG.info("Completed add column family operation on table " + getTableName());
+    } else {
+      LOG.warn("Error on reopening the regions on table " + getTableName());
+    }
+  }
+
+  /**
+   * The procedure could be restarted from a different machine. If the variable is null, we need to
+   * retrieve it.
+   * @return traceEnabled
+   */
+  private Boolean isTraceEnabled() {
+    if (traceEnabled == null) {
+      traceEnabled = LOG.isTraceEnabled();
+    }
+    return traceEnabled;
+  }
+
+  private String getColumnFamilyName() {
+    return cfDescriptor.getNameAsString();
+  }
+
+  /**
+   * Coprocessor Action.
+   * @param env MasterProcedureEnv
+   * @param state the procedure state
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private void runCoprocessorAction(final MasterProcedureEnv env,
+      final ModifyColumnFamilyState 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 MODIFY_COLUMN_FAMILY_PRE_OPERATION:
+            cpHost.preModifyColumnHandler(tableName, cfDescriptor);
+            break;
+          case MODIFY_COLUMN_FAMILY_POST_OPERATION:
+            cpHost.postModifyColumnHandler(tableName, cfDescriptor);
+            break;
+          default:
+            throw new UnsupportedOperationException(this + " unhandled state=" + state);
+          }
+          return null;
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
index a19a975..44b9803 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java
@@ -103,37 +103,6 @@ public class TestTableLockManager {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test(timeout = 600000)
-  public void testLockTimeoutException() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    conf.setInt(TableLockManager.TABLE_WRITE_LOCK_TIMEOUT_MS, 3000);
-    prepareMiniCluster();
-    HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
-    master.getMasterCoprocessorHost().load(TestLockTimeoutExceptionMasterObserver.class,
-        0, TEST_UTIL.getConfiguration());
-
-    ExecutorService executor = Executors.newSingleThreadExecutor();
-    Future<Object> shouldFinish = executor.submit(new Callable<Object>() {
-      @Override
-      public Object call() throws Exception {
-        Admin admin = TEST_UTIL.getHBaseAdmin();
-        admin.deleteColumn(TABLE_NAME, FAMILY);
-        return null;
-      }
-    });
-
-    deleteColumn.await();
-
-    try {
-      Admin admin = TEST_UTIL.getHBaseAdmin();
-      admin.addColumn(TABLE_NAME, new HColumnDescriptor(NEW_FAMILY));
-      fail("Was expecting TableLockTimeoutException");
-    } catch (LockTimeoutException ex) {
-      //expected
-    }
-    shouldFinish.get();
-  }
-
   public static class TestLockTimeoutExceptionMasterObserver extends BaseMasterObserver {
     @Override
     public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
index 5b2f4f6..b5c82e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDeleteFamilyHandler.java
@@ -29,20 +29,22 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -64,10 +66,17 @@ public class TestTableDeleteFamilyHandler {
    */
   @BeforeClass
   public static void beforeAllTests() throws Exception {
-
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
     TEST_UTIL.startMiniCluster(2);
+  }
+
+  @AfterClass
+  public static void afterAllTests() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
 
+  @Before
+  public void setup() throws IOException, InterruptedException {
     // Create a table of three families. This will assign a region.
     TEST_UTIL.createTable(TABLENAME, FAMILIES);
     Table t = TEST_UTIL.getConnection().getTable(TABLENAME);
@@ -86,22 +95,17 @@ public class TestTableDeleteFamilyHandler {
     TEST_UTIL.flush();
 
     t.close();
-  }
 
-  @AfterClass
-  public static void afterAllTests() throws Exception {
-    TEST_UTIL.deleteTable(TABLENAME);
-    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.ensureSomeRegionServersAvailable(2);
   }
 
-  @Before
-  public void setup() throws IOException, InterruptedException {
-    TEST_UTIL.ensureSomeRegionServersAvailable(2);
+  @After
+  public void cleanup() throws Exception {
+    TEST_UTIL.deleteTable(TABLENAME);
   }
 
   @Test
   public void deleteColumnFamilyWithMultipleRegions() throws Exception {
-
     Admin admin = TEST_UTIL.getHBaseAdmin();
     HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
 
@@ -114,7 +118,6 @@ public class TestTableDeleteFamilyHandler {
     assertEquals(3, beforehtd.getColumnFamilies().length);
     HColumnDescriptor[] families = beforehtd.getColumnFamilies();
     for (int i = 0; i < families.length; i++) {
-
       assertTrue(families[i].getNameAsString().equals("cf" + (i + 1)));
     }
 
@@ -179,4 +182,95 @@ public class TestTableDeleteFamilyHandler {
     }
   }
 
+  @Test
+  public void deleteColumnFamilyTwice() throws Exception {
+
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
+    String cfToDelete = "cf1";
+
+    FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
+
+    // 1 - Check if table exists in descriptor
+    assertTrue(admin.isTableAvailable(TABLENAME));
+
+    // 2 - Check if all the target column family exist in descriptor
+    HColumnDescriptor[] families = beforehtd.getColumnFamilies();
+    Boolean foundCF = false;
+    int i;
+    for (i = 0; i < families.length; i++) {
+      if (families[i].getNameAsString().equals(cfToDelete)) {
+        foundCF = true;
+        break;
+      }
+    }
+    assertTrue(foundCF);
+
+    // 3 - Check if table exists in FS
+    Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
+    assertTrue(fs.exists(tableDir));
+
+    // 4 - Check if all the target column family exist in FS
+    FileStatus[] fileStatus = fs.listStatus(tableDir);
+    foundCF = false;
+    for (i = 0; i < fileStatus.length; i++) {
+      if (fileStatus[i].isDirectory() == true) {
+        FileStatus[] cf = fs.listStatus(fileStatus[i].getPath(), new PathFilter() {
+          @Override
+          public boolean accept(Path p) {
+            if (p.getName().contains(HConstants.RECOVERED_EDITS_DIR)) {
+              return false;
+            }
+            return true;
+          }
+        });
+        for (int j = 0; j < cf.length; j++) {
+          if (cf[j].isDirectory() == true && cf[j].getPath().getName().equals(cfToDelete)) {
+            foundCF = true;
+            break;
+          }
+        }
+      }
+      if (foundCF) {
+        break;
+      }
+    }
+    assertTrue(foundCF);
+
+    // TEST - Disable and delete the column family
+    if (admin.isTableEnabled(TABLENAME)) {
+      admin.disableTable(TABLENAME);
+    }
+    admin.deleteColumn(TABLENAME, Bytes.toBytes(cfToDelete));
+
+    // 5 - Check if the target column family is gone from the FS
+    fileStatus = fs.listStatus(tableDir);
+    for (i = 0; i < fileStatus.length; i++) {
+      if (fileStatus[i].isDirectory() == true) {
+        FileStatus[] cf = fs.listStatus(fileStatus[i].getPath(), new PathFilter() {
+          @Override
+          public boolean accept(Path p) {
+            if (WALSplitter.isSequenceIdFile(p)) {
+              return false;
+            }
+            return true;
+          }
+        });
+        for (int j = 0; j < cf.length; j++) {
+          if (cf[j].isDirectory() == true) {
+            assertFalse(cf[j].getPath().getName().equals(cfToDelete));
+          }
+        }
+      }
+    }
+
+    try {
+      // Test: delete again
+      admin.deleteColumn(TABLENAME, Bytes.toBytes(cfToDelete));
+      Assert.fail("Delete a non-exist column family should fail");
+    } catch (InvalidFamilyOperationException e) {
+      // Expected.
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
index 0d51875..c4772ab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestTableDescriptorModification.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -52,7 +54,7 @@ import org.junit.rules.TestName;
  */
 @Category({MasterTests.class, LargeTests.class})
 public class TestTableDescriptorModification {
-  
+
   @Rule public TestName name = new TestName();
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static TableName TABLE_NAME = null;
@@ -74,7 +76,7 @@ public class TestTableDescriptorModification {
     TABLE_NAME = TableName.valueOf(name.getMethodName());
 
   }
-  
+
   @AfterClass
   public static void afterAllTests() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
@@ -124,6 +126,95 @@ public class TestTableDescriptorModification {
   }
 
   @Test
+  public void testAddSameColumnFamilyTwice() throws IOException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    // Create a table with one families
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the table descriptor
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0);
+
+      // Modify the table removing one family and verify the descriptor
+      admin.addColumn(TABLE_NAME, new HColumnDescriptor(FAMILY_1));
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
+
+      try {
+        // Add same column family again - expect failure
+        admin.addColumn(TABLE_NAME, new HColumnDescriptor(FAMILY_1));
+        Assert.fail("Delete a non-exist column family should fail");
+      } catch (InvalidFamilyOperationException e) {
+        // Expected.
+      }
+
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void testModifyColumnFamily() throws IOException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+
+    HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_0);
+    int blockSize = cfDescriptor.getBlocksize();
+    // Create a table with one families
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    baseHtd.addFamily(cfDescriptor);
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the table descriptor
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0);
+
+      int newBlockSize = 2 * blockSize;
+      cfDescriptor.setBlocksize(newBlockSize);
+
+      // Modify colymn family
+      admin.modifyColumn(TABLE_NAME, cfDescriptor);
+
+      HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+      HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
+      assertTrue(hcfd.getBlocksize() == newBlockSize);
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  @Test
+  public void testModifyNonExistingColumnFamily() throws IOException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+
+    HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_1);
+    int blockSize = cfDescriptor.getBlocksize();
+    // Create a table with one families
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the table descriptor
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0);
+
+      int newBlockSize = 2 * blockSize;
+      cfDescriptor.setBlocksize(newBlockSize);
+
+      // Modify a column family that is not in the table.
+      try {
+        admin.modifyColumn(TABLE_NAME, cfDescriptor);
+        Assert.fail("Modify a non-exist column family should fail");
+      } catch (InvalidFamilyOperationException e) {
+        // Expected.
+      }
+
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
+  @Test
   public void testDeleteColumn() throws IOException {
     Admin admin = TEST_UTIL.getHBaseAdmin();
     // Create a table with two families
@@ -144,6 +235,35 @@ public class TestTableDescriptorModification {
     }
   }
 
+  @Test
+  public void testDeleteSameColumnFamilyTwice() throws IOException {
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    // Create a table with two families
+    HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
+    baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
+    baseHtd.addFamily(new HColumnDescriptor(FAMILY_1));
+    admin.createTable(baseHtd);
+    admin.disableTable(TABLE_NAME);
+    try {
+      // Verify the table descriptor
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
+
+      // Modify the table removing one family and verify the descriptor
+      admin.deleteColumn(TABLE_NAME, FAMILY_1);
+      verifyTableDescriptor(TABLE_NAME, FAMILY_0);
+
+      try {
+        // Delete again - expect failure
+        admin.deleteColumn(TABLE_NAME, FAMILY_1);
+        Assert.fail("Delete a non-exist column family should fail");
+      } catch (Exception e) {
+        // Expected.
+      }
+    } finally {
+      admin.deleteTable(TABLE_NAME);
+    }
+  }
+
   private void verifyTableDescriptor(final TableName tableName,
                                      final byte[]... families) throws IOException {
     Admin admin = TEST_UTIL.getHBaseAdmin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/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 d6c19e1..bc97bb9 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
@@ -295,6 +295,40 @@ public class MasterProcedureTestingUtility {
     ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
   }
 
+  public static void validateColumnFamilyAddition(final HMaster master, final TableName tableName,
+      final String family) throws IOException {
+    TableDescriptor htd = master.getTableDescriptors().getDescriptor(tableName);
+    assertTrue(htd != null);
+
+    assertTrue(htd.getHTableDescriptor().hasFamily(family.getBytes()));
+  }
+
+  public static void validateColumnFamilyDeletion(final HMaster master, final TableName tableName,
+      final String family) throws IOException {
+    // verify htd
+    TableDescriptor htd = master.getTableDescriptors().getDescriptor(tableName);
+    assertTrue(htd != null);
+    assertFalse(htd.getHTableDescriptor().hasFamily(family.getBytes()));
+
+    // verify fs
+    final FileSystem fs = master.getMasterFileSystem().getFileSystem();
+    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
+      final Path familyDir = new Path(regionDir, family);
+      assertFalse(family + " family dir should not exist", fs.exists(familyDir));
+    }
+  }
+
+  public static void validateColumnFamilyModification(final HMaster master,
+      final TableName tableName, final String family, HColumnDescriptor columnDescriptor)
+      throws IOException {
+    TableDescriptor htd = master.getTableDescriptors().getDescriptor(tableName);
+    assertTrue(htd != null);
+
+    HColumnDescriptor hcfd = htd.getHTableDescriptor().getFamily(family.getBytes());
+    assertTrue(hcfd.equals(columnDescriptor));
+  }
+
   public static class InjectAbortOnLoadListener
       implements ProcedureExecutor.ProcedureExecutorListener {
     private final ProcedureExecutor<MasterProcedureEnv> procExec;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
new file mode 100644
index 0000000..1490aa1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestAddColumnFamilyProcedure.java
@@ -0,0 +1,246 @@
+/**
+ * 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.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.InvalidFamilyOperationException;
+import org.apache.hadoop.hbase.TableName;
+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.AddColumnFamilyState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestAddColumnFamilyProcedure {
+  private static final Log LOG = LogFactory.getLog(TestAddColumnFamilyProcedure.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 testAddColumnFamily() throws Exception {
+    final TableName tableName = TableName.valueOf("testAddColumnFamily");
+    final String cf1 = "cf1";
+    final String cf2 = "cf2";
+    final HColumnDescriptor columnDescriptor1 = new HColumnDescriptor(cf1);
+    final HColumnDescriptor columnDescriptor2 = new HColumnDescriptor(cf2);
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f3");
+
+    // Test 1: Add a column family online
+    long procId1 =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor1));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf1);
+
+    // Test 2: Add a column family offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    long procId2 =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor2));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
+  }
+
+  @Test(timeout=60000)
+  public void testAddSameColumnFamilyTwice() throws Exception {
+    final TableName tableName = TableName.valueOf("testAddColumnFamilyTwice");
+    final String cf2 = "cf2";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2);
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
+
+    // add the column family
+    long procId1 =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf2);
+
+    // add the column family that exists
+    long procId2 =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+
+    // Second add should fail with InvalidFamilyOperationException
+    ProcedureResult result = procExec.getResult(procId2);
+    assertTrue(result.isFailed());
+    LOG.debug("Add failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+
+    // Do the same add the existing column family - this time offline
+    UTIL.getHBaseAdmin().disableTable(tableName);
+    long procId3 =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId3);
+
+    // Second add should fail with InvalidFamilyOperationException
+    result = procExec.getResult(procId3);
+    assertTrue(result.isFailed());
+    LOG.debug("Add failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecutionOffline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
+    final String cf4 = "cf4";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf4);
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the AddColumnFamily procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = AddColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      AddColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf4);
+  }
+
+  @Test(timeout = 60000)
+  public void testRecoveryAndDoubleExecutionOnline() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
+    final String cf5 = "cf5";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf5);
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3");
+
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the AddColumnFamily procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+
+    // Restart the executor and execute the step twice
+    int numberOfSteps = AddColumnFamilyState.values().length;
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
+      AddColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf5);
+  }
+
+  @Test(timeout = 60000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+    final String cf6 = "cf6";
+    final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf6);
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    // create the table
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the AddColumnFamily procedure && kill the executor
+    long procId =
+        procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
+            columnDescriptor));
+
+    int numberOfSteps = AddColumnFamilyState.values().length - 2; // failing in the middle of proc
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
+      AddColumnFamilyState.values());
+
+    MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
+      tableName, cf6);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}


[03/11] hbase git commit: HBASE-13210 Procedure V2 - master Modify table (Stephen Yuan Jiang)

Posted by mb...@apache.org.
HBASE-13210 Procedure V2 - master Modify table (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: 1aa4d9d545a31c51e270fab023d78a535d1cefb2
Parents: 02c15a2
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:06:30 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 23:51:29 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 1936 ++++++++++++++++--
 .../src/main/protobuf/MasterProcedure.proto     |   18 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   32 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |   10 +-
 .../procedure/MasterDDLOperationHelper.java     |  167 ++
 .../master/procedure/ModifyTableProcedure.java  |  522 +++++
 .../procedure/TestModifyTableProcedure.java     |  403 ++++
 7 files changed, 2915 insertions(+), 173 deletions(-)
----------------------------------------------------------------------



[06/11] hbase git commit: HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/0be8fcc7/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
index 4deb131..3d4d7da 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -371,6 +371,342 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(enum_scope:DeleteTableState)
   }
 
+  /**
+   * Protobuf enum {@code AddColumnFamilyState}
+   */
+  public enum AddColumnFamilyState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>ADD_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    ADD_COLUMN_FAMILY_PREPARE(0, 1),
+    /**
+     * <code>ADD_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    ADD_COLUMN_FAMILY_PRE_OPERATION(1, 2),
+    /**
+     * <code>ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR(2, 3),
+    /**
+     * <code>ADD_COLUMN_FAMILY_POST_OPERATION = 4;</code>
+     */
+    ADD_COLUMN_FAMILY_POST_OPERATION(3, 4),
+    /**
+     * <code>ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;</code>
+     */
+    ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS(4, 5),
+    ;
+
+    /**
+     * <code>ADD_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    public static final int ADD_COLUMN_FAMILY_PREPARE_VALUE = 1;
+    /**
+     * <code>ADD_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    public static final int ADD_COLUMN_FAMILY_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    public static final int ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR_VALUE = 3;
+    /**
+     * <code>ADD_COLUMN_FAMILY_POST_OPERATION = 4;</code>
+     */
+    public static final int ADD_COLUMN_FAMILY_POST_OPERATION_VALUE = 4;
+    /**
+     * <code>ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;</code>
+     */
+    public static final int ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS_VALUE = 5;
+
+
+    public final int getNumber() { return value; }
+
+    public static AddColumnFamilyState valueOf(int value) {
+      switch (value) {
+        case 1: return ADD_COLUMN_FAMILY_PREPARE;
+        case 2: return ADD_COLUMN_FAMILY_PRE_OPERATION;
+        case 3: return ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR;
+        case 4: return ADD_COLUMN_FAMILY_POST_OPERATION;
+        case 5: return ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<AddColumnFamilyState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<AddColumnFamilyState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<AddColumnFamilyState>() {
+            public AddColumnFamilyState findValueByNumber(int number) {
+              return AddColumnFamilyState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(3);
+    }
+
+    private static final AddColumnFamilyState[] VALUES = values();
+
+    public static AddColumnFamilyState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private AddColumnFamilyState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:AddColumnFamilyState)
+  }
+
+  /**
+   * Protobuf enum {@code ModifyColumnFamilyState}
+   */
+  public enum ModifyColumnFamilyState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    MODIFY_COLUMN_FAMILY_PREPARE(0, 1),
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    MODIFY_COLUMN_FAMILY_PRE_OPERATION(1, 2),
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR(2, 3),
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;</code>
+     */
+    MODIFY_COLUMN_FAMILY_POST_OPERATION(3, 4),
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;</code>
+     */
+    MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS(4, 5),
+    ;
+
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    public static final int MODIFY_COLUMN_FAMILY_PREPARE_VALUE = 1;
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    public static final int MODIFY_COLUMN_FAMILY_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    public static final int MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR_VALUE = 3;
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;</code>
+     */
+    public static final int MODIFY_COLUMN_FAMILY_POST_OPERATION_VALUE = 4;
+    /**
+     * <code>MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;</code>
+     */
+    public static final int MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS_VALUE = 5;
+
+
+    public final int getNumber() { return value; }
+
+    public static ModifyColumnFamilyState valueOf(int value) {
+      switch (value) {
+        case 1: return MODIFY_COLUMN_FAMILY_PREPARE;
+        case 2: return MODIFY_COLUMN_FAMILY_PRE_OPERATION;
+        case 3: return MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR;
+        case 4: return MODIFY_COLUMN_FAMILY_POST_OPERATION;
+        case 5: return MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<ModifyColumnFamilyState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<ModifyColumnFamilyState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<ModifyColumnFamilyState>() {
+            public ModifyColumnFamilyState findValueByNumber(int number) {
+              return ModifyColumnFamilyState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(4);
+    }
+
+    private static final ModifyColumnFamilyState[] VALUES = values();
+
+    public static ModifyColumnFamilyState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private ModifyColumnFamilyState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:ModifyColumnFamilyState)
+  }
+
+  /**
+   * Protobuf enum {@code DeleteColumnFamilyState}
+   */
+  public enum DeleteColumnFamilyState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>DELETE_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    DELETE_COLUMN_FAMILY_PREPARE(0, 1),
+    /**
+     * <code>DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    DELETE_COLUMN_FAMILY_PRE_OPERATION(1, 2),
+    /**
+     * <code>DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR(2, 3),
+    /**
+     * <code>DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;</code>
+     */
+    DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT(3, 4),
+    /**
+     * <code>DELETE_COLUMN_FAMILY_POST_OPERATION = 5;</code>
+     */
+    DELETE_COLUMN_FAMILY_POST_OPERATION(4, 5),
+    /**
+     * <code>DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;</code>
+     */
+    DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS(5, 6),
+    ;
+
+    /**
+     * <code>DELETE_COLUMN_FAMILY_PREPARE = 1;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_PREPARE_VALUE = 1;
+    /**
+     * <code>DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR_VALUE = 3;
+    /**
+     * <code>DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT_VALUE = 4;
+    /**
+     * <code>DELETE_COLUMN_FAMILY_POST_OPERATION = 5;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_POST_OPERATION_VALUE = 5;
+    /**
+     * <code>DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;</code>
+     */
+    public static final int DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static DeleteColumnFamilyState valueOf(int value) {
+      switch (value) {
+        case 1: return DELETE_COLUMN_FAMILY_PREPARE;
+        case 2: return DELETE_COLUMN_FAMILY_PRE_OPERATION;
+        case 3: return DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR;
+        case 4: return DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT;
+        case 5: return DELETE_COLUMN_FAMILY_POST_OPERATION;
+        case 6: return DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<DeleteColumnFamilyState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<DeleteColumnFamilyState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<DeleteColumnFamilyState>() {
+            public DeleteColumnFamilyState findValueByNumber(int number) {
+              return DeleteColumnFamilyState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(5);
+    }
+
+    private static final DeleteColumnFamilyState[] VALUES = values();
+
+    public static DeleteColumnFamilyState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private DeleteColumnFamilyState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:DeleteColumnFamilyState)
+  }
+
   public interface CreateTableStateDataOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -4156,66 +4492,4764 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:DeleteTableStateData)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_CreateTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_CreateTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteTableStateData_fieldAccessorTable;
+  public interface AddColumnFamilyStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
-      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
-      "_info\030\001 \002(\0132\020.UserInformation\022\"\n\014table_s" +
-      "chema\030\002 \002(\0132\014.TableSchema\022 \n\013region_info" +
-      "\030\003 \003(\0132\013.RegionInfo\"\341\001\n\024ModifyTableState" +
-      "Data\022#\n\tuser_info\030\001 \002(\0132\020.UserInformatio" +
-      "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
-      "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
-      "\014.TableSchema\022&\n\036delete_column_family_in" +
-      "_modify\030\004 \002(\010\022 \n\013region_info\030\005 \003(\0132\013.Reg",
-      "ionInfo\"}\n\024DeleteTableStateData\022#\n\tuser_" +
-      "info\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_na" +
-      "me\030\002 \002(\0132\n.TableName\022 \n\013region_info\030\003 \003(" +
-      "\0132\013.RegionInfo*\330\001\n\020CreateTableState\022\036\n\032C" +
-      "REATE_TABLE_PRE_OPERATION\020\001\022 \n\034CREATE_TA" +
-      "BLE_WRITE_FS_LAYOUT\020\002\022\034\n\030CREATE_TABLE_AD" +
-      "D_TO_META\020\003\022\037\n\033CREATE_TABLE_ASSIGN_REGIO" +
-      "NS\020\004\022\"\n\036CREATE_TABLE_UPDATE_DESC_CACHE\020\005" +
-      "\022\037\n\033CREATE_TABLE_POST_OPERATION\020\006*\207\002\n\020Mo" +
-      "difyTableState\022\030\n\024MODIFY_TABLE_PREPARE\020\001",
-      "\022\036\n\032MODIFY_TABLE_PRE_OPERATION\020\002\022(\n$MODI" +
-      "FY_TABLE_UPDATE_TABLE_DESCRIPTOR\020\003\022&\n\"MO" +
-      "DIFY_TABLE_REMOVE_REPLICA_COLUMN\020\004\022!\n\035MO" +
-      "DIFY_TABLE_DELETE_FS_LAYOUT\020\005\022\037\n\033MODIFY_" +
-      "TABLE_POST_OPERATION\020\006\022#\n\037MODIFY_TABLE_R" +
-      "EOPEN_ALL_REGIONS\020\007*\337\001\n\020DeleteTableState" +
-      "\022\036\n\032DELETE_TABLE_PRE_OPERATION\020\001\022!\n\035DELE" +
-      "TE_TABLE_REMOVE_FROM_META\020\002\022 \n\034DELETE_TA" +
-      "BLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036DELETE_TABLE_UP" +
-      "DATE_DESC_CACHE\020\004\022!\n\035DELETE_TABLE_UNASSI",
-      "GN_REGIONS\020\005\022\037\n\033DELETE_TABLE_POST_OPERAT" +
-      "ION\020\006BK\n*org.apache.hadoop.hbase.protobu" +
-      "f.generatedB\025MasterProcedureProtosH\001\210\001\001\240" +
-      "\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required .ColumnFamilySchema columnfamily_schema = 3;
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    boolean hasColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder();
+
+    // optional .TableSchema unmodified_table_schema = 4;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    boolean hasUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder();
+
+    // repeated .RegionInfo region_info = 5;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code AddColumnFamilyStateData}
+   */
+  public static final class AddColumnFamilyStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements AddColumnFamilyStateDataOrBuilder {
+    // Use AddColumnFamilyStateData.newBuilder() to construct.
+    private AddColumnFamilyStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private AddColumnFamilyStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final AddColumnFamilyStateData defaultInstance;
+    public static AddColumnFamilyStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public AddColumnFamilyStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private AddColumnFamilyStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = columnfamilySchema_.toBuilder();
+              }
+              columnfamilySchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(columnfamilySchema_);
+                columnfamilySchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000008) == 0x00000008)) {
+                subBuilder = unmodifiedTableSchema_.toBuilder();
+              }
+              unmodifiedTableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(unmodifiedTableSchema_);
+                unmodifiedTableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000008;
+              break;
+            }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<AddColumnFamilyStateData> PARSER =
+        new com.google.protobuf.AbstractParser<AddColumnFamilyStateData>() {
+      public AddColumnFamilyStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new AddColumnFamilyStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<AddColumnFamilyStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required .ColumnFamilySchema columnfamily_schema = 3;
+    public static final int COLUMNFAMILY_SCHEMA_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_;
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public boolean hasColumnfamilySchema() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema() {
+      return columnfamilySchema_;
+    }
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder() {
+      return columnfamilySchema_;
+    }
+
+    // optional .TableSchema unmodified_table_schema = 4;
+    public static final int UNMODIFIED_TABLE_SCHEMA_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public boolean hasUnmodifiedTableSchema() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+      return unmodifiedTableSchema_;
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+      return unmodifiedTableSchema_;
+    }
+
+    // repeated .RegionInfo region_info = 5;
+    public static final int REGION_INFO_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public int getRegionInfoCount() {
+      return regionInfo_.size();
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+      return regionInfo_.get(index);
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index) {
+      return regionInfo_.get(index);
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+      unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      regionInfo_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasColumnfamilySchema()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getColumnfamilySchema().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (hasUnmodifiedTableSchema()) {
+        if (!getUnmodifiedTableSchema().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getRegionInfoCount(); i++) {
+        if (!getRegionInfo(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, columnfamilySchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeMessage(4, unmodifiedTableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        output.writeMessage(5, regionInfo_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, columnfamilySchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, unmodifiedTableSchema_);
+      }
+      for (int i = 0; i < regionInfo_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, regionInfo_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasColumnfamilySchema() == other.hasColumnfamilySchema());
+      if (hasColumnfamilySchema()) {
+        result = result && getColumnfamilySchema()
+            .equals(other.getColumnfamilySchema());
+      }
+      result = result && (hasUnmodifiedTableSchema() == other.hasUnmodifiedTableSchema());
+      if (hasUnmodifiedTableSchema()) {
+        result = result && getUnmodifiedTableSchema()
+            .equals(other.getUnmodifiedTableSchema());
+      }
+      result = result && getRegionInfoList()
+          .equals(other.getRegionInfoList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasColumnfamilySchema()) {
+        hash = (37 * hash) + COLUMNFAMILY_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnfamilySchema().hashCode();
+      }
+      if (hasUnmodifiedTableSchema()) {
+        hash = (37 * hash) + UNMODIFIED_TABLE_SCHEMA_FIELD_NUMBER;
+        hash = (53 * hash) + getUnmodifiedTableSchema().hashCode();
+      }
+      if (getRegionInfoCount() > 0) {
+        hash = (37 * hash) + REGION_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getRegionInfoList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code AddColumnFamilyStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+          getColumnfamilySchemaFieldBuilder();
+          getUnmodifiedTableSchemaFieldBuilder();
+          getRegionInfoFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+        } else {
+          columnfamilySchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+        } else {
+          unmodifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (columnfamilySchemaBuilder_ == null) {
+          result.columnfamilySchema_ = columnfamilySchema_;
+        } else {
+          result.columnfamilySchema_ = columnfamilySchemaBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchema_;
+        } else {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchemaBuilder_.build();
+        }
+        if (regionInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.regionInfo_ = regionInfo_;
+        } else {
+          result.regionInfo_ = regionInfoBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasColumnfamilySchema()) {
+          mergeColumnfamilySchema(other.getColumnfamilySchema());
+        }
+        if (other.hasUnmodifiedTableSchema()) {
+          mergeUnmodifiedTableSchema(other.getUnmodifiedTableSchema());
+        }
+        if (regionInfoBuilder_ == null) {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfo_.isEmpty()) {
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureRegionInfoIsMutable();
+              regionInfo_.addAll(other.regionInfo_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.regionInfo_.isEmpty()) {
+            if (regionInfoBuilder_.isEmpty()) {
+              regionInfoBuilder_.dispose();
+              regionInfoBuilder_ = null;
+              regionInfo_ = other.regionInfo_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              regionInfoBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getRegionInfoFieldBuilder() : null;
+            } else {
+              regionInfoBuilder_.addAllMessages(other.regionInfo_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasColumnfamilySchema()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        if (!getColumnfamilySchema().isInitialized()) {
+          
+          return false;
+        }
+        if (hasUnmodifiedTableSchema()) {
+          if (!getUnmodifiedTableSchema().isInitialized()) {
+            
+            return false;
+          }
+        }
+        for (int i = 0; i < getRegionInfoCount(); i++) {
+          if (!getRegionInfo(i).isInitialized()) {
+            
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required .ColumnFamilySchema columnfamily_schema = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnfamilySchemaBuilder_;
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public boolean hasColumnfamilySchema() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema() {
+        if (columnfamilySchemaBuilder_ == null) {
+          return columnfamilySchema_;
+        } else {
+          return columnfamilySchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder setColumnfamilySchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema value) {
+        if (columnfamilySchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          columnfamilySchema_ = value;
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder setColumnfamilySchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder builderForValue) {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder mergeColumnfamilySchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema value) {
+        if (columnfamilySchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              columnfamilySchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance()) {
+            columnfamilySchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.newBuilder(columnfamilySchema_).mergeFrom(value).buildPartial();
+          } else {
+            columnfamilySchema_ = value;
+          }
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder clearColumnfamilySchema() {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder getColumnfamilySchemaBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getColumnfamilySchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder() {
+        if (columnfamilySchemaBuilder_ != null) {
+          return columnfamilySchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return columnfamilySchema_;
+        }
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
+          getColumnfamilySchemaFieldBuilder() {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
+                  columnfamilySchema_,
+                  getParentForChildren(),
+                  isClean());
+          columnfamilySchema_ = null;
+        }
+        return columnfamilySchemaBuilder_;
+      }
+
+      // optional .TableSchema unmodified_table_schema = 4;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public boolean hasUnmodifiedTableSchema() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          return unmodifiedTableSchema_;
+        } else {
+          return unmodifiedTableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder setUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          unmodifiedTableSchema_ = value;
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder setUnmodifiedTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder mergeUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              unmodifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            unmodifiedTableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(unmodifiedTableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            unmodifiedTableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder clearUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+        if (unmodifiedTableSchemaBuilder_ != null) {
+          return unmodifiedTableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return unmodifiedTableSchema_;
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getUnmodifiedTableSchemaFieldBuilder() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  unmodifiedTableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          unmodifiedTableSchema_ = null;
+        }
+        return unmodifiedTableSchemaBuilder_;
+      }
+
+      // repeated .RegionInfo region_info = 5;
+      private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_ =
+        java.util.Collections.emptyList();
+      private void ensureRegionInfoIsMutable() {
+        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
+          regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>(regionInfo_);
+          bitField0_ |= 0x00000010;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> regionInfoBuilder_;
+
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+        if (regionInfoBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        } else {
+          return regionInfoBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public int getRegionInfoCount() {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.size();
+        } else {
+          return regionInfoBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);
+        } else {
+          return regionInfoBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder setRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder addRegionInfo(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo value) {
+        if (regionInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, value);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder addRegionInfo(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder addRegionInfo(
+          int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder builderForValue) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          regionInfoBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder addAllRegionInfo(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> values) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          super.addAll(values, regionInfo_);
+          onChanged();
+        } else {
+          regionInfoBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder clearRegionInfo() {
+        if (regionInfoBuilder_ == null) {
+          regionInfo_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+          onChanged();
+        } else {
+          regionInfoBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public Builder removeRegionInfo(int index) {
+        if (regionInfoBuilder_ == null) {
+          ensureRegionInfoIsMutable();
+          regionInfo_.remove(index);
+          onChanged();
+        } else {
+          regionInfoBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder getRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+          int index) {
+        if (regionInfoBuilder_ == null) {
+          return regionInfo_.get(index);  } else {
+          return regionInfoBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+           getRegionInfoOrBuilderList() {
+        if (regionInfoBuilder_ != null) {
+          return regionInfoBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(regionInfo_);
+        }
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder() {
+        return getRegionInfoFieldBuilder().addBuilder(
+            org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder addRegionInfoBuilder(
+          int index) {
+        return getRegionInfoFieldBuilder().addBuilder(
+            index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .RegionInfo region_info = 5;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder> 
+           getRegionInfoBuilderList() {
+        return getRegionInfoFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+          getRegionInfoFieldBuilder() {
+        if (regionInfoBuilder_ == null) {
+          regionInfoBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder>(
+                  regionInfo_,
+                  ((bitField0_ & 0x00000010) == 0x00000010),
+                  getParentForChildren(),
+                  isClean());
+          regionInfo_ = null;
+        }
+        return regionInfoBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:AddColumnFamilyStateData)
+    }
+
+    static {
+      defaultInstance = new AddColumnFamilyStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:AddColumnFamilyStateData)
+  }
+
+  public interface ModifyColumnFamilyStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required .ColumnFamilySchema columnfamily_schema = 3;
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    boolean hasColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder();
+
+    // optional .TableSchema unmodified_table_schema = 4;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    boolean hasUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder();
+
+    // repeated .RegionInfo region_info = 5;
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code ModifyColumnFamilyStateData}
+   */
+  public static final class ModifyColumnFamilyStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements ModifyColumnFamilyStateDataOrBuilder {
+    // Use ModifyColumnFamilyStateData.newBuilder() to construct.
+    private ModifyColumnFamilyStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ModifyColumnFamilyStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ModifyColumnFamilyStateData defaultInstance;
+    public static ModifyColumnFamilyStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ModifyColumnFamilyStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ModifyColumnFamilyStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.

<TRUNCATED>

[07/11] hbase git commit: HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)

Posted by mb...@apache.org.
HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: 0be8fcc78fe9d37ee1e25778013ebeb3dd65191b
Parents: 1aa4d9d
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:21:18 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 23:58:16 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 5170 +++++++++++++++++-
 .../src/main/protobuf/MasterProcedure.proto     |   49 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   30 +-
 .../handler/TableDeleteFamilyHandler.java       |    6 +-
 .../procedure/AddColumnFamilyProcedure.java     |  416 ++
 .../procedure/DeleteColumnFamilyProcedure.java  |  448 ++
 .../procedure/ModifyColumnFamilyProcedure.java  |  400 ++
 .../hbase/master/TestTableLockManager.java      |   31 -
 .../handler/TestTableDeleteFamilyHandler.java   |  122 +-
 .../TestTableDescriptorModification.java        |  124 +-
 .../MasterProcedureTestingUtility.java          |   34 +
 .../procedure/TestAddColumnFamilyProcedure.java |  246 +
 .../TestDeleteColumnFamilyProcedure.java        |  302 +
 .../TestModifyColumnFamilyProcedure.java        |  238 +
 14 files changed, 7499 insertions(+), 117 deletions(-)
----------------------------------------------------------------------



[11/11] hbase git commit: HBASE-13290 Procedure v2 - client enable/disable table sync (Stephen Yuan Jiang)

Posted by mb...@apache.org.
HBASE-13290 Procedure v2 - client enable/disable table sync (Stephen Yuan Jiang)


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

Branch: refs/heads/hbase-12439
Commit: 9d919c9168c2087f6a31e74d8195e3dcc466cb35
Parents: ec87cf1
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:53:51 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 23:58:34 2015 +0100

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 230 ++++++--
 .../hbase/protobuf/generated/MasterProtos.java  | 583 ++++++++++++-------
 hbase-protocol/src/main/protobuf/Master.proto   |   2 +
 .../org/apache/hadoop/hbase/master/HMaster.java |  10 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   8 +-
 .../hadoop/hbase/master/MasterServices.java     |   4 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   8 +-
 7 files changed, 586 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 1697c03..b37bc77 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -101,8 +101,10 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotReq
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
@@ -944,12 +946,20 @@ public class HBaseAdmin implements Admin {
   @Override
   public void enableTable(final TableName tableName)
   throws IOException {
-    enableTableAsync(tableName);
-
-    // Wait until all regions are enabled
-    waitUntilTableIsEnabled(tableName);
-
-    LOG.info("Enabled table " + tableName);
+    Future<Void> future = enableTableAsyncV2(tableName);
+    try {
+      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(e);
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException)e.getCause();
+      } else {
+        throw new IOException(e.getCause());
+      }
+    }
   }
 
   public void enableTable(final byte[] tableName)
@@ -1016,16 +1026,7 @@ public class HBaseAdmin implements Admin {
   @Override
   public void enableTableAsync(final TableName tableName)
   throws IOException {
-    TableName.isLegalFullyQualifiedTableName(tableName.getName());
-    executeCallable(new MasterCallable<Void>(getConnection()) {
-      @Override
-      public Void call(int callTimeout) throws ServiceException {
-        LOG.info("Started enable of " + tableName);
-        EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
-        master.enableTable(null,req);
-        return null;
-      }
-    });
+    enableTableAsyncV2(tableName);
   }
 
   public void enableTableAsync(final byte[] tableName)
@@ -1039,6 +1040,84 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
+   * Enable the table but does not block and wait for it be completely enabled.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async enable. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
+  // TODO: This should be called Async but it will break binary compatibility
+  private Future<Void> enableTableAsyncV2(final TableName tableName) throws IOException {
+    TableName.isLegalFullyQualifiedTableName(tableName.getName());
+    EnableTableResponse response = executeCallable(
+        new MasterCallable<EnableTableResponse>(getConnection()) {
+      @Override
+      public EnableTableResponse call(int callTimeout) throws ServiceException {
+        LOG.info("Started enable of " + tableName);
+        EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
+        return master.enableTable(null,req);
+      }
+    });
+    return new EnableTableFuture(this, tableName, response);
+  }
+
+  private static class EnableTableFuture extends ProcedureFuture<Void> {
+    private final TableName tableName;
+
+    public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
+        final EnableTableResponse response) {
+      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
+      this.tableName = tableName;
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitTableEnabled(deadlineTs);
+      return null;
+    }
+
+    @Override
+    protected Void postOperationResult(final Void result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      LOG.info("Enabled " + tableName);
+      return result;
+    }
+
+    private void waitTableEnabled(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForState(deadlineTs, new WaitForStateCallable() {
+        @Override
+        public boolean checkState(int tries) throws IOException {
+          boolean enabled;
+          try {
+            enabled = getAdmin().isTableEnabled(tableName);
+          } catch (TableNotFoundException tnfe) {
+            return false;
+          }
+          return enabled && getAdmin().isTableAvailable(tableName);
+        }
+
+        @Override
+        public void throwInterruptedException() throws InterruptedIOException {
+          throw new InterruptedIOException("Interrupted when waiting for table to be enabled");
+        }
+
+        @Override
+        public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+          throw new TimeoutException("Table " + tableName + " not yet enabled after " +
+              elapsedTime + "msec");
+        }
+      });
+    }
+  }
+
+  /**
    * Enable tables matching the passed in pattern and wait on completion.
    *
    * Warning: Use this method carefully, there is no prompting and the effect is
@@ -1096,16 +1175,7 @@ public class HBaseAdmin implements Admin {
    */
   @Override
   public void disableTableAsync(final TableName tableName) throws IOException {
-    TableName.isLegalFullyQualifiedTableName(tableName.getName());
-    executeCallable(new MasterCallable<Void>(getConnection()) {
-      @Override
-      public Void call(int callTimeout) throws ServiceException {
-        LOG.info("Started disable of " + tableName);
-        DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
-        master.disableTable(null,req);
-        return null;
-      }
-    });
+    disableTableAsyncV2(tableName);
   }
 
   public void disableTableAsync(final byte[] tableName) throws IOException {
@@ -1130,32 +1200,20 @@ public class HBaseAdmin implements Admin {
   @Override
   public void disableTable(final TableName tableName)
   throws IOException {
-    disableTableAsync(tableName);
-    // Wait until table is disabled
-    boolean disabled = false;
-    for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
-      disabled = isTableDisabled(tableName);
-      if (disabled) {
-        break;
-      }
-      long sleep = getPauseTime(tries);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
-          "disabled in " + tableName);
-      }
-      try {
-        Thread.sleep(sleep);
-      } catch (InterruptedException e) {
-        // Do this conversion rather than let it out because do not want to
-        // change the method signature.
-        throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
+    Future<Void> future = disableTableAsyncV2(tableName);
+    try {
+      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(e);
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException)e.getCause();
+      } else {
+        throw new IOException(e.getCause());
       }
     }
-    if (!disabled) {
-      throw new RegionException("Retries exhausted, it took too long to wait"+
-        " for the table " + tableName + " to be disabled.");
-    }
-    LOG.info("Disabled " + tableName);
   }
 
   public void disableTable(final byte[] tableName)
@@ -1169,6 +1227,78 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
+   * Disable the table but does not block and wait for it be completely disabled.
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
+   * It may throw ExecutionException if there was an error while executing the operation
+   * or TimeoutException in case the wait timeout was not long enough to allow the
+   * operation to complete.
+   *
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async disable. You can use Future.get(long, TimeUnit)
+   *    to wait on the operation to complete.
+   */
+  // TODO: This should be called Async but it will break binary compatibility
+  private Future<Void> disableTableAsyncV2(final TableName tableName) throws IOException {
+    TableName.isLegalFullyQualifiedTableName(tableName.getName());
+    DisableTableResponse response = executeCallable(
+        new MasterCallable<DisableTableResponse>(getConnection()) {
+      @Override
+      public DisableTableResponse call(int callTimeout) throws ServiceException {
+        LOG.info("Started disable of " + tableName);
+        DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
+        return master.disableTable(null, req);
+      }
+    });
+    return new DisableTableFuture(this, tableName, response);
+  }
+
+  private static class DisableTableFuture extends ProcedureFuture<Void> {
+    private final TableName tableName;
+
+    public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
+        final DisableTableResponse response) {
+      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
+      this.tableName = tableName;
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitTableDisabled(deadlineTs);
+      return null;
+    }
+
+    @Override
+    protected Void postOperationResult(final Void result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      LOG.info("Disabled " + tableName);
+      return result;
+    }
+
+    private void waitTableDisabled(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForState(deadlineTs, new WaitForStateCallable() {
+        @Override
+        public boolean checkState(int tries) throws IOException {
+          return getAdmin().isTableDisabled(tableName);
+        }
+
+        @Override
+        public void throwInterruptedException() throws InterruptedIOException {
+          throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
+        }
+
+        @Override
+        public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+          throw new TimeoutException("Table " + tableName + " not yet disabled after " +
+              elapsedTime + "msec");
+        }
+      });
+    }
+  }
+
+  /**
    * Disable tables matching the passed in pattern and wait on completion.
    *
    * Warning: Use this method carefully, there is no prompting and the effect is

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index cc6f201..463f82f 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -12053,6 +12053,16 @@ public final class MasterProtos {
 
   public interface EnableTableResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint64 proc_id = 1;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    long getProcId();
   }
   /**
    * Protobuf type {@code EnableTableResponse}
@@ -12087,6 +12097,7 @@ public final class MasterProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -12104,6 +12115,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -12143,7 +12159,25 @@ public final class MasterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
     private void initFields() {
+      procId_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -12157,6 +12191,9 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, procId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -12166,6 +12203,10 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, procId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -12189,6 +12230,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -12202,6 +12248,10 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -12311,6 +12361,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -12337,6 +12389,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.procId_ = procId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -12352,6 +12411,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -12377,6 +12439,40 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:EnableTableResponse)
     }
@@ -12952,6 +13048,16 @@ public final class MasterProtos {
 
   public interface DisableTableResponseOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
+
+    // optional uint64 proc_id = 1;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    long getProcId();
   }
   /**
    * Protobuf type {@code DisableTableResponse}
@@ -12986,6 +13092,7 @@ public final class MasterProtos {
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
+      int mutable_bitField0_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -13003,6 +13110,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -13042,7 +13154,25 @@ public final class MasterProtos {
       return PARSER;
     }
 
+    private int bitField0_;
+    // optional uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
     private void initFields() {
+      procId_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -13056,6 +13186,9 @@ public final class MasterProtos {
     public void writeTo(com.google.protobuf.CodedOutputStream output)
                         throws java.io.IOException {
       getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, procId_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -13065,6 +13198,10 @@ public final class MasterProtos {
       if (size != -1) return size;
 
       size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, procId_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -13088,6 +13225,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -13101,6 +13243,10 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -13210,6 +13356,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -13236,6 +13384,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.procId_ = procId_;
+        result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
       }
@@ -13251,6 +13406,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -13276,6 +13434,40 @@ public final class MasterProtos {
         }
         return this;
       }
+      private int bitField0_;
+
+      // optional uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
 
       // @@protoc_insertion_point(builder_scope:DisableTableResponse)
     }
@@ -52960,199 +53152,200 @@ public final class MasterProtos {
       "e\030\001 \002(\0132\n.TableName\022\035\n\016preserveSplits\030\002 " +
       "\001(\010:\005false\"\027\n\025TruncateTableResponse\"4\n\022E" +
       "nableTableRequest\022\036\n\ntable_name\030\001 \002(\0132\n." +
-      "TableName\"\025\n\023EnableTableResponse\"5\n\023Disa" +
-      "bleTableRequest\022\036\n\ntable_name\030\001 \002(\0132\n.Ta" +
-      "bleName\"\026\n\024DisableTableResponse\"X\n\022Modif" +
-      "yTableRequest\022\036\n\ntable_name\030\001 \002(\0132\n.Tabl" +
-      "eName\022\"\n\014table_schema\030\002 \002(\0132\014.TableSchem" +
-      "a\"\025\n\023ModifyTableResponse\"K\n\026CreateNamesp",
-      "aceRequest\0221\n\023namespaceDescriptor\030\001 \002(\0132" +
-      "\024.NamespaceDescriptor\"\031\n\027CreateNamespace" +
-      "Response\"/\n\026DeleteNamespaceRequest\022\025\n\rna" +
-      "mespaceName\030\001 \002(\t\"\031\n\027DeleteNamespaceResp" +
-      "onse\"K\n\026ModifyNamespaceRequest\0221\n\023namesp" +
-      "aceDescriptor\030\001 \002(\0132\024.NamespaceDescripto" +
-      "r\"\031\n\027ModifyNamespaceResponse\"6\n\035GetNames" +
-      "paceDescriptorRequest\022\025\n\rnamespaceName\030\001" +
-      " \002(\t\"S\n\036GetNamespaceDescriptorResponse\0221" +
-      "\n\023namespaceDescriptor\030\001 \002(\0132\024.NamespaceD",
-      "escriptor\"!\n\037ListNamespaceDescriptorsReq" +
-      "uest\"U\n ListNamespaceDescriptorsResponse" +
-      "\0221\n\023namespaceDescriptor\030\001 \003(\0132\024.Namespac" +
-      "eDescriptor\"?\n&ListTableDescriptorsByNam" +
-      "espaceRequest\022\025\n\rnamespaceName\030\001 \002(\t\"L\n\'" +
-      "ListTableDescriptorsByNamespaceResponse\022" +
-      "!\n\013tableSchema\030\001 \003(\0132\014.TableSchema\"9\n Li" +
-      "stTableNamesByNamespaceRequest\022\025\n\rnamesp" +
-      "aceName\030\001 \002(\t\"B\n!ListTableNamesByNamespa" +
-      "ceResponse\022\035\n\ttableName\030\001 \003(\0132\n.TableNam",
-      "e\"\021\n\017ShutdownRequest\"\022\n\020ShutdownResponse" +
-      "\"\023\n\021StopMasterRequest\"\024\n\022StopMasterRespo" +
-      "nse\"\020\n\016BalanceRequest\"\'\n\017BalanceResponse" +
-      "\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031SetBalancerRun" +
-      "ningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002" +
-      " \001(\010\"8\n\032SetBalancerRunningResponse\022\032\n\022pr" +
-      "ev_balance_value\030\001 \001(\010\"\032\n\030IsBalancerEnab" +
-      "ledRequest\",\n\031IsBalancerEnabledResponse\022" +
-      "\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCatalogScanReques" +
-      "t\"-\n\026RunCatalogScanResponse\022\023\n\013scan_resu",
-      "lt\030\001 \001(\005\"-\n\033EnableCatalogJanitorRequest\022" +
-      "\016\n\006enable\030\001 \002(\010\"2\n\034EnableCatalogJanitorR" +
-      "esponse\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalog" +
-      "JanitorEnabledRequest\"0\n\037IsCatalogJanito" +
-      "rEnabledResponse\022\r\n\005value\030\001 \002(\010\"9\n\017Snaps" +
-      "hotRequest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotD" +
-      "escription\",\n\020SnapshotResponse\022\030\n\020expect" +
-      "ed_timeout\030\001 \002(\003\"\036\n\034GetCompletedSnapshot" +
-      "sRequest\"H\n\035GetCompletedSnapshotsRespons" +
-      "e\022\'\n\tsnapshots\030\001 \003(\0132\024.SnapshotDescripti",
-      "on\"?\n\025DeleteSnapshotRequest\022&\n\010snapshot\030" +
-      "\001 \002(\0132\024.SnapshotDescription\"\030\n\026DeleteSna" +
-      "pshotResponse\"@\n\026RestoreSnapshotRequest\022" +
-      "&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescription\"" +
-      "\031\n\027RestoreSnapshotResponse\"?\n\025IsSnapshot" +
-      "DoneRequest\022&\n\010snapshot\030\001 \001(\0132\024.Snapshot" +
-      "Description\"U\n\026IsSnapshotDoneResponse\022\023\n" +
-      "\004done\030\001 \001(\010:\005false\022&\n\010snapshot\030\002 \001(\0132\024.S" +
-      "napshotDescription\"F\n\034IsRestoreSnapshotD" +
-      "oneRequest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotD",
-      "escription\"4\n\035IsRestoreSnapshotDoneRespo" +
-      "nse\022\023\n\004done\030\001 \001(\010:\005false\"=\n\033GetSchemaAlt" +
-      "erStatusRequest\022\036\n\ntable_name\030\001 \002(\0132\n.Ta" +
-      "bleName\"T\n\034GetSchemaAlterStatusResponse\022" +
-      "\035\n\025yet_to_update_regions\030\001 \001(\r\022\025\n\rtotal_" +
-      "regions\030\002 \001(\r\"\202\001\n\032GetTableDescriptorsReq" +
-      "uest\022\037\n\013table_names\030\001 \003(\0132\n.TableName\022\r\n" +
-      "\005regex\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010" +
-      ":\005false\022\021\n\tnamespace\030\004 \001(\t\"A\n\033GetTableDe" +
-      "scriptorsResponse\022\"\n\014table_schema\030\001 \003(\0132",
-      "\014.TableSchema\"[\n\024GetTableNamesRequest\022\r\n" +
-      "\005regex\030\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010" +
-      ":\005false\022\021\n\tnamespace\030\003 \001(\t\"8\n\025GetTableNa" +
-      "mesResponse\022\037\n\013table_names\030\001 \003(\0132\n.Table" +
-      "Name\"6\n\024GetTableStateRequest\022\036\n\ntable_na" +
-      "me\030\001 \002(\0132\n.TableName\"9\n\025GetTableStateRes" +
-      "ponse\022 \n\013table_state\030\001 \002(\0132\013.TableState\"" +
-      "\031\n\027GetClusterStatusRequest\"B\n\030GetCluster" +
-      "StatusResponse\022&\n\016cluster_status\030\001 \002(\0132\016" +
-      ".ClusterStatus\"\030\n\026IsMasterRunningRequest",
-      "\"4\n\027IsMasterRunningResponse\022\031\n\021is_master" +
-      "_running\030\001 \002(\010\"@\n\024ExecProcedureRequest\022(" +
-      "\n\tprocedure\030\001 \002(\0132\025.ProcedureDescription" +
-      "\"F\n\025ExecProcedureResponse\022\030\n\020expected_ti" +
-      "meout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"B\n\026IsPr" +
-      "ocedureDoneRequest\022(\n\tprocedure\030\001 \001(\0132\025." +
-      "ProcedureDescription\"W\n\027IsProcedureDoneR" +
-      "esponse\022\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snapshot" +
-      "\030\002 \001(\0132\025.ProcedureDescription\",\n\031GetProc" +
-      "edureResultRequest\022\017\n\007proc_id\030\001 \002(\004\"\347\001\n\032",
-      "GetProcedureResultResponse\0220\n\005state\030\001 \002(" +
-      "\0162!.GetProcedureResultResponse.State\022\022\n\n" +
-      "start_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n" +
-      "\006result\030\004 \001(\014\022+\n\texception\030\005 \001(\0132\030.Forei" +
-      "gnExceptionMessage\"1\n\005State\022\r\n\tNOT_FOUND" +
-      "\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"\273\001\n\017SetQuo" +
-      "taRequest\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_gro" +
-      "up\030\002 \001(\t\022\021\n\tnamespace\030\003 \001(\t\022\036\n\ntable_nam" +
-      "e\030\004 \001(\0132\n.TableName\022\022\n\nremove_all\030\005 \001(\010\022" +
-      "\026\n\016bypass_globals\030\006 \001(\010\022\"\n\010throttle\030\007 \001(",
-      "\0132\020.ThrottleRequest\"\022\n\020SetQuotaResponse\"" +
-      "A\n\037MajorCompactionTimestampRequest\022\036\n\nta" +
-      "ble_name\030\001 \002(\0132\n.TableName\"L\n(MajorCompa" +
-      "ctionTimestampForRegionRequest\022 \n\006region" +
-      "\030\001 \002(\0132\020.RegionSpecifier\"@\n MajorCompact" +
-      "ionTimestampResponse\022\034\n\024compaction_times" +
-      "tamp\030\001 \002(\0032\343\033\n\rMasterService\022S\n\024GetSchem" +
-      "aAlterStatus\022\034.GetSchemaAlterStatusReque" +
-      "st\032\035.GetSchemaAlterStatusResponse\022P\n\023Get" +
-      "TableDescriptors\022\033.GetTableDescriptorsRe",
-      "quest\032\034.GetTableDescriptorsResponse\022>\n\rG" +
-      "etTableNames\022\025.GetTableNamesRequest\032\026.Ge" +
-      "tTableNamesResponse\022G\n\020GetClusterStatus\022" +
-      "\030.GetClusterStatusRequest\032\031.GetClusterSt" +
-      "atusResponse\022D\n\017IsMasterRunning\022\027.IsMast" +
-      "erRunningRequest\032\030.IsMasterRunningRespon" +
-      "se\0222\n\tAddColumn\022\021.AddColumnRequest\032\022.Add" +
-      "ColumnResponse\022;\n\014DeleteColumn\022\024.DeleteC" +
-      "olumnRequest\032\025.DeleteColumnResponse\022;\n\014M" +
-      "odifyColumn\022\024.ModifyColumnRequest\032\025.Modi",
-      "fyColumnResponse\0225\n\nMoveRegion\022\022.MoveReg" +
-      "ionRequest\032\023.MoveRegionResponse\022Y\n\026Dispa" +
-      "tchMergingRegions\022\036.DispatchMergingRegio" +
-      "nsRequest\032\037.DispatchMergingRegionsRespon" +
-      "se\022;\n\014AssignRegion\022\024.AssignRegionRequest" +
-      "\032\025.AssignRegionResponse\022A\n\016UnassignRegio" +
-      "n\022\026.UnassignRegionRequest\032\027.UnassignRegi" +
-      "onResponse\022>\n\rOfflineRegion\022\025.OfflineReg" +
-      "ionRequest\032\026.OfflineRegionResponse\0228\n\013De" +
-      "leteTable\022\023.DeleteTableRequest\032\024.DeleteT",
-      "ableResponse\022>\n\rtruncateTable\022\025.Truncate" +
-      "TableRequest\032\026.TruncateTableResponse\0228\n\013" +
-      "EnableTable\022\023.EnableTableRequest\032\024.Enabl" +
-      "eTableResponse\022;\n\014DisableTable\022\024.Disable" +
-      "TableRequest\032\025.DisableTableResponse\0228\n\013M" +
-      "odifyTable\022\023.ModifyTableRequest\032\024.Modify" +
-      "TableResponse\0228\n\013CreateTable\022\023.CreateTab" +
-      "leRequest\032\024.CreateTableResponse\022/\n\010Shutd" +
-      "own\022\020.ShutdownRequest\032\021.ShutdownResponse" +
-      "\0225\n\nStopMaster\022\022.StopMasterRequest\032\023.Sto",
-      "pMasterResponse\022,\n\007Balance\022\017.BalanceRequ" +
-      "est\032\020.BalanceResponse\022M\n\022SetBalancerRunn" +
-      "ing\022\032.SetBalancerRunningRequest\032\033.SetBal" +
-      "ancerRunningResponse\022J\n\021IsBalancerEnable" +
-      "d\022\031.IsBalancerEnabledRequest\032\032.IsBalance" +
-      "rEnabledResponse\022A\n\016RunCatalogScan\022\026.Run" +
-      "CatalogScanRequest\032\027.RunCatalogScanRespo" +
-      "nse\022S\n\024EnableCatalogJanitor\022\034.EnableCata" +
-      "logJanitorRequest\032\035.EnableCatalogJanitor" +
-      "Response\022\\\n\027IsCatalogJanitorEnabled\022\037.Is",
-      "CatalogJanitorEnabledRequest\032 .IsCatalog" +
-      "JanitorEnabledResponse\022L\n\021ExecMasterServ" +
-      "ice\022\032.CoprocessorServiceRequest\032\033.Coproc" +
-      "essorServiceResponse\022/\n\010Snapshot\022\020.Snaps" +
-      "hotRequest\032\021.SnapshotResponse\022V\n\025GetComp" +
-      "letedSnapshots\022\035.GetCompletedSnapshotsRe" +
-      "quest\032\036.GetCompletedSnapshotsResponse\022A\n" +
-      "\016DeleteSnapshot\022\026.DeleteSnapshotRequest\032" +
-      "\027.DeleteSnapshotResponse\022A\n\016IsSnapshotDo" +
-      "ne\022\026.IsSnapshotDoneRequest\032\027.IsSnapshotD",
-      "oneResponse\022D\n\017RestoreSnapshot\022\027.Restore" +
-      "SnapshotRequest\032\030.RestoreSnapshotRespons" +
-      "e\022V\n\025IsRestoreSnapshotDone\022\035.IsRestoreSn" +
-      "apshotDoneRequest\032\036.IsRestoreSnapshotDon" +
-      "eResponse\022>\n\rExecProcedure\022\025.ExecProcedu" +
-      "reRequest\032\026.ExecProcedureResponse\022E\n\024Exe" +
-      "cProcedureWithRet\022\025.ExecProcedureRequest" +
-      "\032\026.ExecProcedureResponse\022D\n\017IsProcedureD" +
-      "one\022\027.IsProcedureDoneRequest\032\030.IsProcedu" +
-      "reDoneResponse\022D\n\017ModifyNamespace\022\027.Modi",
-      "fyNamespaceRequest\032\030.ModifyNamespaceResp" +
-      "onse\022D\n\017CreateNamespace\022\027.CreateNamespac" +
-      "eRequest\032\030.CreateNamespaceResponse\022D\n\017De" +
-      "leteNamespace\022\027.DeleteNamespaceRequest\032\030" +
-      ".DeleteNamespaceResponse\022Y\n\026GetNamespace" +
-      "Descriptor\022\036.GetNamespaceDescriptorReque" +
-      "st\032\037.GetNamespaceDescriptorResponse\022_\n\030L" +
-      "istNamespaceDescriptors\022 .ListNamespaceD" +
-      "escriptorsRequest\032!.ListNamespaceDescrip" +
-      "torsResponse\022t\n\037ListTableDescriptorsByNa",
-      "mespace\022\'.ListTableDescriptorsByNamespac" +
-      "eRequest\032(.ListTableDescriptorsByNamespa" +
-      "ceResponse\022b\n\031ListTableNamesByNamespace\022" +
-      "!.ListTableNamesByNamespaceRequest\032\".Lis" +
-      "tTableNamesByNamespaceResponse\022>\n\rGetTab" +
-      "leState\022\025.GetTableStateRequest\032\026.GetTabl" +
-      "eStateResponse\022/\n\010SetQuota\022\020.SetQuotaReq" +
-      "uest\032\021.SetQuotaResponse\022f\n\037getLastMajorC" +
-      "ompactionTimestamp\022 .MajorCompactionTime" +
-      "stampRequest\032!.MajorCompactionTimestampR",
-      "esponse\022x\n(getLastMajorCompactionTimesta" +
-      "mpForRegion\022).MajorCompactionTimestampFo" +
-      "rRegionRequest\032!.MajorCompactionTimestam" +
-      "pResponse\022M\n\022getProcedureResult\022\032.GetPro" +
-      "cedureResultRequest\032\033.GetProcedureResult" +
-      "ResponseBB\n*org.apache.hadoop.hbase.prot" +
-      "obuf.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
+      "TableName\"&\n\023EnableTableResponse\022\017\n\007proc" +
+      "_id\030\001 \001(\004\"5\n\023DisableTableRequest\022\036\n\ntabl" +
+      "e_name\030\001 \002(\0132\n.TableName\"\'\n\024DisableTable" +
+      "Response\022\017\n\007proc_id\030\001 \001(\004\"X\n\022ModifyTable" +
+      "Request\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022" +
+      "\"\n\014table_schema\030\002 \002(\0132\014.TableSchema\"\025\n\023M",
+      "odifyTableResponse\"K\n\026CreateNamespaceReq" +
+      "uest\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.Name" +
+      "spaceDescriptor\"\031\n\027CreateNamespaceRespon" +
+      "se\"/\n\026DeleteNamespaceRequest\022\025\n\rnamespac" +
+      "eName\030\001 \002(\t\"\031\n\027DeleteNamespaceResponse\"K" +
+      "\n\026ModifyNamespaceRequest\0221\n\023namespaceDes" +
+      "criptor\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027M" +
+      "odifyNamespaceResponse\"6\n\035GetNamespaceDe" +
+      "scriptorRequest\022\025\n\rnamespaceName\030\001 \002(\t\"S" +
+      "\n\036GetNamespaceDescriptorResponse\0221\n\023name",
+      "spaceDescriptor\030\001 \002(\0132\024.NamespaceDescrip" +
+      "tor\"!\n\037ListNamespaceDescriptorsRequest\"U" +
+      "\n ListNamespaceDescriptorsResponse\0221\n\023na" +
+      "mespaceDescriptor\030\001 \003(\0132\024.NamespaceDescr" +
+      "iptor\"?\n&ListTableDescriptorsByNamespace" +
+      "Request\022\025\n\rnamespaceName\030\001 \002(\t\"L\n\'ListTa" +
+      "bleDescriptorsByNamespaceResponse\022!\n\013tab" +
+      "leSchema\030\001 \003(\0132\014.TableSchema\"9\n ListTabl" +
+      "eNamesByNamespaceRequest\022\025\n\rnamespaceNam" +
+      "e\030\001 \002(\t\"B\n!ListTableNamesByNamespaceResp",
+      "onse\022\035\n\ttableName\030\001 \003(\0132\n.TableName\"\021\n\017S" +
+      "hutdownRequest\"\022\n\020ShutdownResponse\"\023\n\021St" +
+      "opMasterRequest\"\024\n\022StopMasterResponse\"\020\n" +
+      "\016BalanceRequest\"\'\n\017BalanceResponse\022\024\n\014ba" +
+      "lancer_ran\030\001 \002(\010\"<\n\031SetBalancerRunningRe" +
+      "quest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8" +
+      "\n\032SetBalancerRunningResponse\022\032\n\022prev_bal" +
+      "ance_value\030\001 \001(\010\"\032\n\030IsBalancerEnabledReq" +
+      "uest\",\n\031IsBalancerEnabledResponse\022\017\n\007ena" +
+      "bled\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026R",
+      "unCatalogScanResponse\022\023\n\013scan_result\030\001 \001" +
+      "(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006ena" +
+      "ble\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespons" +
+      "e\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanito" +
+      "rEnabledRequest\"0\n\037IsCatalogJanitorEnabl" +
+      "edResponse\022\r\n\005value\030\001 \002(\010\"9\n\017SnapshotReq" +
+      "uest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescrip" +
+      "tion\",\n\020SnapshotResponse\022\030\n\020expected_tim" +
+      "eout\030\001 \002(\003\"\036\n\034GetCompletedSnapshotsReque" +
+      "st\"H\n\035GetCompletedSnapshotsResponse\022\'\n\ts",
+      "napshots\030\001 \003(\0132\024.SnapshotDescription\"?\n\025" +
+      "DeleteSnapshotRequest\022&\n\010snapshot\030\001 \002(\0132" +
+      "\024.SnapshotDescription\"\030\n\026DeleteSnapshotR" +
+      "esponse\"@\n\026RestoreSnapshotRequest\022&\n\010sna" +
+      "pshot\030\001 \002(\0132\024.SnapshotDescription\"\031\n\027Res" +
+      "toreSnapshotResponse\"?\n\025IsSnapshotDoneRe" +
+      "quest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescri" +
+      "ption\"U\n\026IsSnapshotDoneResponse\022\023\n\004done\030" +
+      "\001 \001(\010:\005false\022&\n\010snapshot\030\002 \001(\0132\024.Snapsho" +
+      "tDescription\"F\n\034IsRestoreSnapshotDoneReq",
+      "uest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescrip" +
+      "tion\"4\n\035IsRestoreSnapshotDoneResponse\022\023\n" +
+      "\004done\030\001 \001(\010:\005false\"=\n\033GetSchemaAlterStat" +
+      "usRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableNam" +
+      "e\"T\n\034GetSchemaAlterStatusResponse\022\035\n\025yet" +
+      "_to_update_regions\030\001 \001(\r\022\025\n\rtotal_region" +
+      "s\030\002 \001(\r\"\202\001\n\032GetTableDescriptorsRequest\022\037" +
+      "\n\013table_names\030\001 \003(\0132\n.TableName\022\r\n\005regex" +
+      "\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010:\005fals" +
+      "e\022\021\n\tnamespace\030\004 \001(\t\"A\n\033GetTableDescript",
+      "orsResponse\022\"\n\014table_schema\030\001 \003(\0132\014.Tabl" +
+      "eSchema\"[\n\024GetTableNamesRequest\022\r\n\005regex" +
+      "\030\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010:\005fals" +
+      "e\022\021\n\tnamespace\030\003 \001(\t\"8\n\025GetTableNamesRes" +
+      "ponse\022\037\n\013table_names\030\001 \003(\0132\n.TableName\"6" +
+      "\n\024GetTableStateRequest\022\036\n\ntable_name\030\001 \002" +
+      "(\0132\n.TableName\"9\n\025GetTableStateResponse\022" +
+      " \n\013table_state\030\001 \002(\0132\013.TableState\"\031\n\027Get" +
+      "ClusterStatusRequest\"B\n\030GetClusterStatus" +
+      "Response\022&\n\016cluster_status\030\001 \002(\0132\016.Clust",
+      "erStatus\"\030\n\026IsMasterRunningRequest\"4\n\027Is" +
+      "MasterRunningResponse\022\031\n\021is_master_runni" +
+      "ng\030\001 \002(\010\"@\n\024ExecProcedureRequest\022(\n\tproc" +
+      "edure\030\001 \002(\0132\025.ProcedureDescription\"F\n\025Ex" +
+      "ecProcedureResponse\022\030\n\020expected_timeout\030" +
+      "\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"B\n\026IsProcedur" +
+      "eDoneRequest\022(\n\tprocedure\030\001 \001(\0132\025.Proced" +
+      "ureDescription\"W\n\027IsProcedureDoneRespons" +
+      "e\022\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snapshot\030\002 \001(\013" +
+      "2\025.ProcedureDescription\",\n\031GetProcedureR",
+      "esultRequest\022\017\n\007proc_id\030\001 \002(\004\"\347\001\n\032GetPro" +
+      "cedureResultResponse\0220\n\005state\030\001 \002(\0162!.Ge" +
+      "tProcedureResultResponse.State\022\022\n\nstart_" +
+      "time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006resul" +
+      "t\030\004 \001(\014\022+\n\texception\030\005 \001(\0132\030.ForeignExce" +
+      "ptionMessage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007" +
+      "RUNNING\020\001\022\014\n\010FINISHED\020\002\"\273\001\n\017SetQuotaRequ" +
+      "est\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001" +
+      "(\t\022\021\n\tnamespace\030\003 \001(\t\022\036\n\ntable_name\030\004 \001(" +
+      "\0132\n.TableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016byp",
+      "ass_globals\030\006 \001(\010\022\"\n\010throttle\030\007 \001(\0132\020.Th" +
+      "rottleRequest\"\022\n\020SetQuotaResponse\"A\n\037Maj" +
+      "orCompactionTimestampRequest\022\036\n\ntable_na" +
+      "me\030\001 \002(\0132\n.TableName\"L\n(MajorCompactionT" +
+      "imestampForRegionRequest\022 \n\006region\030\001 \002(\013" +
+      "2\020.RegionSpecifier\"@\n MajorCompactionTim" +
+      "estampResponse\022\034\n\024compaction_timestamp\030\001" +
+      " \002(\0032\343\033\n\rMasterService\022S\n\024GetSchemaAlter" +
+      "Status\022\034.GetSchemaAlterStatusRequest\032\035.G" +
+      "etSchemaAlterStatusResponse\022P\n\023GetTableD",
+      "escriptors\022\033.GetTableDescriptorsRequest\032" +
+      "\034.GetTableDescriptorsResponse\022>\n\rGetTabl" +
+      "eNames\022\025.GetTableNamesRequest\032\026.GetTable" +
+      "NamesResponse\022G\n\020GetClusterStatus\022\030.GetC" +
+      "lusterStatusRequest\032\031.GetClusterStatusRe" +
+      "sponse\022D\n\017IsMasterRunning\022\027.IsMasterRunn" +
+      "ingRequest\032\030.IsMasterRunningResponse\0222\n\t" +
+      "AddColumn\022\021.AddColumnRequest\032\022.AddColumn" +
+      "Response\022;\n\014DeleteColumn\022\024.DeleteColumnR" +
+      "equest\032\025.DeleteColumnResponse\022;\n\014ModifyC",
+      "olumn\022\024.ModifyColumnRequest\032\025.ModifyColu" +
+      "mnResponse\0225\n\nMoveRegion\022\022.MoveRegionReq" +
+      "uest\032\023.MoveRegionResponse\022Y\n\026DispatchMer" +
+      "gingRegions\022\036.DispatchMergingRegionsRequ" +
+      "est\032\037.DispatchMergingRegionsResponse\022;\n\014" +
+      "AssignRegion\022\024.AssignRegionRequest\032\025.Ass" +
+      "ignRegionResponse\022A\n\016UnassignRegion\022\026.Un" +
+      "assignRegionRequest\032\027.UnassignRegionResp" +
+      "onse\022>\n\rOfflineRegion\022\025.OfflineRegionReq" +
+      "uest\032\026.OfflineRegionResponse\0228\n\013DeleteTa",
+      "ble\022\023.DeleteTableRequest\032\024.DeleteTableRe" +
+      "sponse\022>\n\rtruncateTable\022\025.TruncateTableR" +
+      "equest\032\026.TruncateTableResponse\0228\n\013Enable" +
+      "Table\022\023.EnableTableRequest\032\024.EnableTable" +
+      "Response\022;\n\014DisableTable\022\024.DisableTableR" +
+      "equest\032\025.DisableTableResponse\0228\n\013ModifyT" +
+      "able\022\023.ModifyTableRequest\032\024.ModifyTableR" +
+      "esponse\0228\n\013CreateTable\022\023.CreateTableRequ" +
+      "est\032\024.CreateTableResponse\022/\n\010Shutdown\022\020." +
+      "ShutdownRequest\032\021.ShutdownResponse\0225\n\nSt",
+      "opMaster\022\022.StopMasterRequest\032\023.StopMaste" +
+      "rResponse\022,\n\007Balance\022\017.BalanceRequest\032\020." +
+      "BalanceResponse\022M\n\022SetBalancerRunning\022\032." +
+      "SetBalancerRunningRequest\032\033.SetBalancerR" +
+      "unningResponse\022J\n\021IsBalancerEnabled\022\031.Is" +
+      "BalancerEnabledRequest\032\032.IsBalancerEnabl" +
+      "edResponse\022A\n\016RunCatalogScan\022\026.RunCatalo" +
+      "gScanRequest\032\027.RunCatalogScanResponse\022S\n" +
+      "\024EnableCatalogJanitor\022\034.EnableCatalogJan" +
+      "itorRequest\032\035.EnableCatalogJanitorRespon",
+      "se\022\\\n\027IsCatalogJanitorEnabled\022\037.IsCatalo" +
+      "gJanitorEnabledRequest\032 .IsCatalogJanito" +
+      "rEnabledResponse\022L\n\021ExecMasterService\022\032." +
+      "CoprocessorServiceRequest\032\033.CoprocessorS" +
+      "erviceResponse\022/\n\010Snapshot\022\020.SnapshotReq" +
+      "uest\032\021.SnapshotResponse\022V\n\025GetCompletedS" +
+      "napshots\022\035.GetCompletedSnapshotsRequest\032" +
+      "\036.GetCompletedSnapshotsResponse\022A\n\016Delet" +
+      "eSnapshot\022\026.DeleteSnapshotRequest\032\027.Dele" +
+      "teSnapshotResponse\022A\n\016IsSnapshotDone\022\026.I",
+      "sSnapshotDoneRequest\032\027.IsSnapshotDoneRes" +
+      "ponse\022D\n\017RestoreSnapshot\022\027.RestoreSnapsh" +
+      "otRequest\032\030.RestoreSnapshotResponse\022V\n\025I" +
+      "sRestoreSnapshotDone\022\035.IsRestoreSnapshot" +
+      "DoneRequest\032\036.IsRestoreSnapshotDoneRespo" +
+      "nse\022>\n\rExecProcedure\022\025.ExecProcedureRequ" +
+      "est\032\026.ExecProcedureResponse\022E\n\024ExecProce" +
+      "dureWithRet\022\025.ExecProcedureRequest\032\026.Exe" +
+      "cProcedureResponse\022D\n\017IsProcedureDone\022\027." +
+      "IsProcedureDoneRequest\032\030.IsProcedureDone",
+      "Response\022D\n\017ModifyNamespace\022\027.ModifyName" +
+      "spaceRequest\032\030.ModifyNamespaceResponse\022D" +
+      "\n\017CreateNamespace\022\027.CreateNamespaceReque" +
+      "st\032\030.CreateNamespaceResponse\022D\n\017DeleteNa" +
+      "mespace\022\027.DeleteNamespaceRequest\032\030.Delet" +
+      "eNamespaceResponse\022Y\n\026GetNamespaceDescri" +
+      "ptor\022\036.GetNamespaceDescriptorRequest\032\037.G" +
+      "etNamespaceDescriptorResponse\022_\n\030ListNam" +
+      "espaceDescriptors\022 .ListNamespaceDescrip" +
+      "torsRequest\032!.ListNamespaceDescriptorsRe",
+      "sponse\022t\n\037ListTableDescriptorsByNamespac" +
+      "e\022\'.ListTableDescriptorsByNamespaceReque" +
+      "st\032(.ListTableDescriptorsByNamespaceResp" +
+      "onse\022b\n\031ListTableNamesByNamespace\022!.List" +
+      "TableNamesByNamespaceRequest\032\".ListTable" +
+      "NamesByNamespaceResponse\022>\n\rGetTableStat" +
+      "e\022\025.GetTableStateRequest\032\026.GetTableState" +
+      "Response\022/\n\010SetQuota\022\020.SetQuotaRequest\032\021" +
+      ".SetQuotaResponse\022f\n\037getLastMajorCompact" +
+      "ionTimestamp\022 .MajorCompactionTimestampR",
+      "equest\032!.MajorCompactionTimestampRespons" +
+      "e\022x\n(getLastMajorCompactionTimestampForR" +
+      "egion\022).MajorCompactionTimestampForRegio" +
+      "nRequest\032!.MajorCompactionTimestampRespo" +
+      "nse\022M\n\022getProcedureResult\022\032.GetProcedure" +
+      "ResultRequest\032\033.GetProcedureResultRespon" +
+      "seBB\n*org.apache.hadoop.hbase.protobuf.g" +
+      "eneratedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -53302,7 +53495,7 @@ public final class MasterProtos {
           internal_static_EnableTableResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_EnableTableResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_DisableTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(24);
           internal_static_DisableTableRequest_fieldAccessorTable = new
@@ -53314,7 +53507,7 @@ public final class MasterProtos {
           internal_static_DisableTableResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DisableTableResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_ModifyTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(26);
           internal_static_ModifyTableRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index c30d92a..d5f4275 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -133,6 +133,7 @@ message EnableTableRequest {
 }
 
 message EnableTableResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DisableTableRequest {
@@ -140,6 +141,7 @@ message DisableTableRequest {
 }
 
 message DisableTableResponse {
+  optional uint64 proc_id = 1;
 }
 
 message ModifyTableRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/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 ff28081..fdbc31c 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
@@ -1675,7 +1675,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void enableTable(final TableName tableName) throws IOException {
+  public long enableTable(final TableName tableName) throws IOException {
     checkInitialized();
     if (cpHost != null) {
       cpHost.preEnableTable(tableName);
@@ -1697,12 +1697,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.postEnableTable(tableName);
     }
 
-    // TODO: return procId as part of client-side change
-    // return procId;
+    return procId;
   }
 
   @Override
-  public void disableTable(final TableName tableName) throws IOException {
+  public long disableTable(final TableName tableName) throws IOException {
     checkInitialized();
     if (cpHost != null) {
       cpHost.preDisableTable(tableName);
@@ -1725,8 +1724,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.postDisableTable(tableName);
     }
 
-    // TODO: return procId as part of client-side change
-    // return procId;
+    return procId;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index abdbf5a..91c406c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -489,11 +489,11 @@ public class MasterRpcServices extends RSRpcServices
   public DisableTableResponse disableTable(RpcController controller,
       DisableTableRequest request) throws ServiceException {
     try {
-      master.disableTable(ProtobufUtil.toTableName(request.getTableName()));
+      long procId = master.disableTable(ProtobufUtil.toTableName(request.getTableName()));
+      return DisableTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
-    return DisableTableResponse.newBuilder().build();
   }
 
   @Override
@@ -575,11 +575,11 @@ public class MasterRpcServices extends RSRpcServices
   public EnableTableResponse enableTable(RpcController controller,
       EnableTableRequest request) throws ServiceException {
     try {
-      master.enableTable(ProtobufUtil.toTableName(request.getTableName()));
+      long procId = master.enableTable(ProtobufUtil.toTableName(request.getTableName()));
+      return EnableTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
-    return EnableTableResponse.newBuilder().build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index d823b35..6153139 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -137,14 +137,14 @@ public interface MasterServices extends Server {
    * @param tableName The table name
    * @throws IOException
    */
-  void enableTable(final TableName tableName) throws IOException;
+  long enableTable(final TableName tableName) throws IOException;
 
   /**
    * Disable an existing table
    * @param tableName The table name
    * @throws IOException
    */
-  void disableTable(final TableName tableName) throws IOException;
+  long disableTable(final TableName tableName) throws IOException;
 
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/9d919c91/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 5b2e50d..00cad06 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -441,10 +441,14 @@ public class TestCatalogJanitor {
         throws IOException { }
 
     @Override
-    public void enableTable(TableName tableName) throws IOException { }
+    public long enableTable(TableName tableName) throws IOException {
+      return -1;
+    }
 
     @Override
-    public void disableTable(TableName tableName) throws IOException { }
+    public long disableTable(TableName tableName) throws IOException {
+      return -1;
+    }
 
     @Override
     public void addColumn(TableName tableName, HColumnDescriptor column)


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

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/ec87cf1b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
index 3d4d7da..9334160 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -707,6 +707,242 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(enum_scope:DeleteColumnFamilyState)
   }
 
+  /**
+   * Protobuf enum {@code EnableTableState}
+   */
+  public enum EnableTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>ENABLE_TABLE_PREPARE = 1;</code>
+     */
+    ENABLE_TABLE_PREPARE(0, 1),
+    /**
+     * <code>ENABLE_TABLE_PRE_OPERATION = 2;</code>
+     */
+    ENABLE_TABLE_PRE_OPERATION(1, 2),
+    /**
+     * <code>ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;</code>
+     */
+    ENABLE_TABLE_SET_ENABLING_TABLE_STATE(2, 3),
+    /**
+     * <code>ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;</code>
+     */
+    ENABLE_TABLE_MARK_REGIONS_ONLINE(3, 4),
+    /**
+     * <code>ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;</code>
+     */
+    ENABLE_TABLE_SET_ENABLED_TABLE_STATE(4, 5),
+    /**
+     * <code>ENABLE_TABLE_POST_OPERATION = 6;</code>
+     */
+    ENABLE_TABLE_POST_OPERATION(5, 6),
+    ;
+
+    /**
+     * <code>ENABLE_TABLE_PREPARE = 1;</code>
+     */
+    public static final int ENABLE_TABLE_PREPARE_VALUE = 1;
+    /**
+     * <code>ENABLE_TABLE_PRE_OPERATION = 2;</code>
+     */
+    public static final int ENABLE_TABLE_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;</code>
+     */
+    public static final int ENABLE_TABLE_SET_ENABLING_TABLE_STATE_VALUE = 3;
+    /**
+     * <code>ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;</code>
+     */
+    public static final int ENABLE_TABLE_MARK_REGIONS_ONLINE_VALUE = 4;
+    /**
+     * <code>ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;</code>
+     */
+    public static final int ENABLE_TABLE_SET_ENABLED_TABLE_STATE_VALUE = 5;
+    /**
+     * <code>ENABLE_TABLE_POST_OPERATION = 6;</code>
+     */
+    public static final int ENABLE_TABLE_POST_OPERATION_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static EnableTableState valueOf(int value) {
+      switch (value) {
+        case 1: return ENABLE_TABLE_PREPARE;
+        case 2: return ENABLE_TABLE_PRE_OPERATION;
+        case 3: return ENABLE_TABLE_SET_ENABLING_TABLE_STATE;
+        case 4: return ENABLE_TABLE_MARK_REGIONS_ONLINE;
+        case 5: return ENABLE_TABLE_SET_ENABLED_TABLE_STATE;
+        case 6: return ENABLE_TABLE_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<EnableTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<EnableTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<EnableTableState>() {
+            public EnableTableState findValueByNumber(int number) {
+              return EnableTableState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(6);
+    }
+
+    private static final EnableTableState[] VALUES = values();
+
+    public static EnableTableState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private EnableTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:EnableTableState)
+  }
+
+  /**
+   * Protobuf enum {@code DisableTableState}
+   */
+  public enum DisableTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>DISABLE_TABLE_PREPARE = 1;</code>
+     */
+    DISABLE_TABLE_PREPARE(0, 1),
+    /**
+     * <code>DISABLE_TABLE_PRE_OPERATION = 2;</code>
+     */
+    DISABLE_TABLE_PRE_OPERATION(1, 2),
+    /**
+     * <code>DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;</code>
+     */
+    DISABLE_TABLE_SET_DISABLING_TABLE_STATE(2, 3),
+    /**
+     * <code>DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;</code>
+     */
+    DISABLE_TABLE_MARK_REGIONS_OFFLINE(3, 4),
+    /**
+     * <code>DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;</code>
+     */
+    DISABLE_TABLE_SET_DISABLED_TABLE_STATE(4, 5),
+    /**
+     * <code>DISABLE_TABLE_POST_OPERATION = 6;</code>
+     */
+    DISABLE_TABLE_POST_OPERATION(5, 6),
+    ;
+
+    /**
+     * <code>DISABLE_TABLE_PREPARE = 1;</code>
+     */
+    public static final int DISABLE_TABLE_PREPARE_VALUE = 1;
+    /**
+     * <code>DISABLE_TABLE_PRE_OPERATION = 2;</code>
+     */
+    public static final int DISABLE_TABLE_PRE_OPERATION_VALUE = 2;
+    /**
+     * <code>DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;</code>
+     */
+    public static final int DISABLE_TABLE_SET_DISABLING_TABLE_STATE_VALUE = 3;
+    /**
+     * <code>DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;</code>
+     */
+    public static final int DISABLE_TABLE_MARK_REGIONS_OFFLINE_VALUE = 4;
+    /**
+     * <code>DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;</code>
+     */
+    public static final int DISABLE_TABLE_SET_DISABLED_TABLE_STATE_VALUE = 5;
+    /**
+     * <code>DISABLE_TABLE_POST_OPERATION = 6;</code>
+     */
+    public static final int DISABLE_TABLE_POST_OPERATION_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static DisableTableState valueOf(int value) {
+      switch (value) {
+        case 1: return DISABLE_TABLE_PREPARE;
+        case 2: return DISABLE_TABLE_PRE_OPERATION;
+        case 3: return DISABLE_TABLE_SET_DISABLING_TABLE_STATE;
+        case 4: return DISABLE_TABLE_MARK_REGIONS_OFFLINE;
+        case 5: return DISABLE_TABLE_SET_DISABLED_TABLE_STATE;
+        case 6: return DISABLE_TABLE_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<DisableTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<DisableTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<DisableTableState>() {
+            public DisableTableState findValueByNumber(int number) {
+              return DisableTableState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.getDescriptor().getEnumTypes().get(7);
+    }
+
+    private static final DisableTableState[] VALUES = values();
+
+    public static DisableTableState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private DisableTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:DisableTableState)
+  }
+
   public interface CreateTableStateDataOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -9140,120 +9376,1900 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:DeleteColumnFamilyStateData)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_CreateTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_CreateTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_AddColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_AddColumnFamilyStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyColumnFamilyStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteColumnFamilyStateData_fieldAccessorTable;
+  public interface EnableTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required bool skip_table_state_check = 3;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean hasSkipTableStateCheck();
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean getSkipTableStateCheck();
   }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
-      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
-      "_info\030\001 \002(\0132\020.UserInformation\022\"\n\014table_s" +
-      "chema\030\002 \002(\0132\014.TableSchema\022 \n\013region_info" +
-      "\030\003 \003(\0132\013.RegionInfo\"\341\001\n\024ModifyTableState" +
-      "Data\022#\n\tuser_info\030\001 \002(\0132\020.UserInformatio" +
-      "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
-      "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
-      "\014.TableSchema\022&\n\036delete_column_family_in" +
-      "_modify\030\004 \002(\010\022 \n\013region_info\030\005 \003(\0132\013.Reg",
-      "ionInfo\"}\n\024DeleteTableStateData\022#\n\tuser_" +
-      "info\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_na" +
-      "me\030\002 \002(\0132\n.TableName\022 \n\013region_info\030\003 \003(" +
-      "\0132\013.RegionInfo\"\342\001\n\030AddColumnFamilyStateD" +
-      "ata\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation" +
-      "\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\0220\n\023colu" +
-      "mnfamily_schema\030\003 \002(\0132\023.ColumnFamilySche" +
-      "ma\022-\n\027unmodified_table_schema\030\004 \001(\0132\014.Ta" +
-      "bleSchema\022 \n\013region_info\030\005 \003(\0132\013.RegionI" +
-      "nfo\"\345\001\n\033ModifyColumnFamilyStateData\022#\n\tu",
-      "ser_info\030\001 \002(\0132\020.UserInformation\022\036\n\ntabl" +
-      "e_name\030\002 \002(\0132\n.TableName\0220\n\023columnfamily" +
-      "_schema\030\003 \002(\0132\023.ColumnFamilySchema\022-\n\027un" +
-      "modified_table_schema\030\004 \001(\0132\014.TableSchem" +
-      "a\022 \n\013region_info\030\005 \003(\0132\013.RegionInfo\"\316\001\n\033" +
-      "DeleteColumnFamilyStateData\022#\n\tuser_info" +
-      "\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_name\030\002" +
-      " \002(\0132\n.TableName\022\031\n\021columnfamily_name\030\003 " +
-      "\002(\014\022-\n\027unmodified_table_schema\030\004 \001(\0132\014.T" +
-      "ableSchema\022 \n\013region_info\030\005 \003(\0132\013.Region",
-      "Info*\330\001\n\020CreateTableState\022\036\n\032CREATE_TABL" +
-      "E_PRE_OPERATION\020\001\022 \n\034CREATE_TABLE_WRITE_" +
-      "FS_LAYOUT\020\002\022\034\n\030CREATE_TABLE_ADD_TO_META\020" +
-      "\003\022\037\n\033CREATE_TABLE_ASSIGN_REGIONS\020\004\022\"\n\036CR" +
-      "EATE_TABLE_UPDATE_DESC_CACHE\020\005\022\037\n\033CREATE" +
-      "_TABLE_POST_OPERATION\020\006*\207\002\n\020ModifyTableS" +
-      "tate\022\030\n\024MODIFY_TABLE_PREPARE\020\001\022\036\n\032MODIFY" +
-      "_TABLE_PRE_OPERATION\020\002\022(\n$MODIFY_TABLE_U" +
-      "PDATE_TABLE_DESCRIPTOR\020\003\022&\n\"MODIFY_TABLE" +
-      "_REMOVE_REPLICA_COLUMN\020\004\022!\n\035MODIFY_TABLE",
-      "_DELETE_FS_LAYOUT\020\005\022\037\n\033MODIFY_TABLE_POST" +
-      "_OPERATION\020\006\022#\n\037MODIFY_TABLE_REOPEN_ALL_" +
-      "REGIONS\020\007*\337\001\n\020DeleteTableState\022\036\n\032DELETE" +
-      "_TABLE_PRE_OPERATION\020\001\022!\n\035DELETE_TABLE_R" +
-      "EMOVE_FROM_META\020\002\022 \n\034DELETE_TABLE_CLEAR_" +
-      "FS_LAYOUT\020\003\022\"\n\036DELETE_TABLE_UPDATE_DESC_" +
-      "CACHE\020\004\022!\n\035DELETE_TABLE_UNASSIGN_REGIONS" +
-      "\020\005\022\037\n\033DELETE_TABLE_POST_OPERATION\020\006*\331\001\n\024" +
-      "AddColumnFamilyState\022\035\n\031ADD_COLUMN_FAMIL" +
-      "Y_PREPARE\020\001\022#\n\037ADD_COLUMN_FAMILY_PRE_OPE",
-      "RATION\020\002\022-\n)ADD_COLUMN_FAMILY_UPDATE_TAB" +
-      "LE_DESCRIPTOR\020\003\022$\n ADD_COLUMN_FAMILY_POS" +
-      "T_OPERATION\020\004\022(\n$ADD_COLUMN_FAMILY_REOPE" +
-      "N_ALL_REGIONS\020\005*\353\001\n\027ModifyColumnFamilySt" +
-      "ate\022 \n\034MODIFY_COLUMN_FAMILY_PREPARE\020\001\022&\n" +
-      "\"MODIFY_COLUMN_FAMILY_PRE_OPERATION\020\002\0220\n" +
-      ",MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCR" +
-      "IPTOR\020\003\022\'\n#MODIFY_COLUMN_FAMILY_POST_OPE" +
-      "RATION\020\004\022+\n\'MODIFY_COLUMN_FAMILY_REOPEN_" +
-      "ALL_REGIONS\020\005*\226\002\n\027DeleteColumnFamilyStat",
-      "e\022 \n\034DELETE_COLUMN_FAMILY_PREPARE\020\001\022&\n\"D" +
-      "ELETE_COLUMN_FAMILY_PRE_OPERATION\020\002\0220\n,D" +
-      "ELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIP" +
-      "TOR\020\003\022)\n%DELETE_COLUMN_FAMILY_DELETE_FS_" +
-      "LAYOUT\020\004\022\'\n#DELETE_COLUMN_FAMILY_POST_OP" +
-      "ERATION\020\005\022+\n\'DELETE_COLUMN_FAMILY_REOPEN" +
-      "_ALL_REGIONS\020\006BK\n*org.apache.hadoop.hbas" +
-      "e.protobuf.generatedB\025MasterProcedurePro" +
-      "tosH\001\210\001\001\240\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
-        public com.google.protobuf.ExtensionRegistry assignDescriptors(
-            com.google.protobuf.Descriptors.FileDescriptor root) {
-          descriptor = root;
+  /**
+   * Protobuf type {@code EnableTableStateData}
+   */
+  public static final class EnableTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements EnableTableStateDataOrBuilder {
+    // Use EnableTableStateData.newBuilder() to construct.
+    private EnableTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private EnableTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final EnableTableStateData defaultInstance;
+    public static EnableTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public EnableTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private EnableTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              skipTableStateCheck_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<EnableTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<EnableTableStateData>() {
+      public EnableTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new EnableTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<EnableTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required bool skip_table_state_check = 3;
+    public static final int SKIP_TABLE_STATE_CHECK_FIELD_NUMBER = 3;
+    private boolean skipTableStateCheck_;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean hasSkipTableStateCheck() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean getSkipTableStateCheck() {
+      return skipTableStateCheck_;
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      skipTableStateCheck_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSkipTableStateCheck()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, skipTableStateCheck_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, skipTableStateCheck_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasSkipTableStateCheck() == other.hasSkipTableStateCheck());
+      if (hasSkipTableStateCheck()) {
+        result = result && (getSkipTableStateCheck()
+            == other.getSkipTableStateCheck());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasSkipTableStateCheck()) {
+        hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipTableStateCheck());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code EnableTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        skipTableStateCheck_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.skipTableStateCheck_ = skipTableStateCheck_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasSkipTableStateCheck()) {
+          setSkipTableStateCheck(other.getSkipTableStateCheck());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasSkipTableStateCheck()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required bool skip_table_state_check = 3;
+      private boolean skipTableStateCheck_ ;
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean hasSkipTableStateCheck() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean getSkipTableStateCheck() {
+        return skipTableStateCheck_;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder setSkipTableStateCheck(boolean value) {
+        bitField0_ |= 0x00000004;
+        skipTableStateCheck_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder clearSkipTableStateCheck() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        skipTableStateCheck_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:EnableTableStateData)
+    }
+
+    static {
+      defaultInstance = new EnableTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:EnableTableStateData)
+  }
+
+  public interface DisableTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required bool skip_table_state_check = 3;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean hasSkipTableStateCheck();
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean getSkipTableStateCheck();
+  }
+  /**
+   * Protobuf type {@code DisableTableStateData}
+   */
+  public static final class DisableTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements DisableTableStateDataOrBuilder {
+    // Use DisableTableStateData.newBuilder() to construct.
+    private DisableTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private DisableTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final DisableTableStateData defaultInstance;
+    public static DisableTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public DisableTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private DisableTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              skipTableStateCheck_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<DisableTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<DisableTableStateData>() {
+      public DisableTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new DisableTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<DisableTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required bool skip_table_state_check = 3;
+    public static final int SKIP_TABLE_STATE_CHECK_FIELD_NUMBER = 3;
+    private boolean skipTableStateCheck_;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean hasSkipTableStateCheck() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean getSkipTableStateCheck() {
+      return skipTableStateCheck_;
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      skipTableStateCheck_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSkipTableStateCheck()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, skipTableStateCheck_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, skipTableStateCheck_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasSkipTableStateCheck() == other.hasSkipTableStateCheck());
+      if (hasSkipTableStateCheck()) {
+        result = result && (getSkipTableStateCheck()
+            == other.getSkipTableStateCheck());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasSkipTableStateCheck()) {
+        hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipTableStateCheck());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code DisableTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        skipTableStateCheck_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.skipTableStateCheck_ = skipTableStateCheck_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasSkipTableStateCheck()) {
+          setSkipTableStateCheck(other.getSkipTableStateCheck());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasSkipTableStateCheck()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required bool skip_table_state_check = 3;
+      private boolean skipTableStateCheck_ ;
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean hasSkipTableStateCheck() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean getSkipTableStateCheck() {
+        return skipTableStateCheck_;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder setSkipTableStateCheck(boolean value) {
+        bitField0_ |= 0x00000004;
+        skipTableStateCheck_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder clearSkipTableStateCheck() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        skipTableStateCheck_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:DisableTableStateData)
+    }
+
+    static {
+      defaultInstance = new DisableTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:DisableTableStateData)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CreateTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CreateTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_AddColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_AddColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EnableTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EnableTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DisableTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DisableTableStateData_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
+      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
+      "

<TRUNCATED>