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

[01/18] hbase git commit: HBASE-13203 Procedure v2 - master create/delete table

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
new file mode 100644
index 0000000..903dbd3
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -0,0 +1,179 @@
+/**
+ * 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.InterruptedIOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoordinatedStateException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
+import org.apache.hadoop.hbase.ServerName;
+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.exceptions.TimeoutIOException;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.ServerManager;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+
+/**
+ * Helper to synchronously wait on conditions.
+ * This will be removed in the future (mainly when the AssignmentManager will be
+ * replaced with a Procedure version) by using ProcedureYieldException,
+ * and the queue will handle waiting and scheduling based on events.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class ProcedureSyncWait {
+  private static final Log LOG = LogFactory.getLog(ProcedureSyncWait.class);
+
+  private ProcedureSyncWait() {}
+
+  @InterfaceAudience.Private
+  public interface Predicate<T> {
+    T evaluate() throws IOException;
+  }
+
+  public static byte[] submitAndWaitProcedure(ProcedureExecutor<MasterProcedureEnv> procExec,
+      final Procedure proc) throws IOException {
+    long procId = procExec.submitProcedure(proc);
+    return waitForProcedureToComplete(procExec, procId);
+  }
+
+  public static byte[] waitForProcedureToComplete(ProcedureExecutor<MasterProcedureEnv> procExec,
+      final long procId) throws IOException {
+    while (!procExec.isFinished(procId) && procExec.isRunning()) {
+      // TODO: add a config to make it tunable
+      // Dev Consideration: are we waiting forever, or we can set up some timeout value?
+      Threads.sleepWithoutInterrupt(250);
+    }
+    ProcedureResult result = procExec.getResult(procId);
+    if (result != null) {
+      if (result.isFailed()) {
+        // If the procedure fails, we should always have an exception captured. Throw it.
+        throw result.getException().unwrapRemoteException();
+      }
+      return result.getResult();
+    } else {
+      if (procExec.isRunning()) {
+        throw new IOException("Procedure " + procId + "not found");
+      } else {
+        throw new IOException("The Master is Aborting");
+      }
+    }
+  }
+
+  public static <T> T waitFor(MasterProcedureEnv env, String purpose, Predicate<T> predicate)
+      throws IOException {
+    final Configuration conf = env.getMasterConfiguration();
+    final long waitTime = conf.getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
+    final long waitingTimeForEvents = conf.getInt("hbase.master.event.waiting.time", 1000);
+    return waitFor(env, waitTime, waitingTimeForEvents, purpose, predicate);
+  }
+
+  public static <T> T waitFor(MasterProcedureEnv env, long waitTime, long waitingTimeForEvents,
+      String purpose, Predicate<T> predicate) throws IOException {
+    final long done = EnvironmentEdgeManager.currentTime() + waitTime;
+    do {
+      T result = predicate.evaluate();
+      if (result != null && !result.equals(Boolean.FALSE)) {
+        return result;
+      }
+      try {
+        Thread.sleep(waitingTimeForEvents);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while sleeping, waiting on " + purpose);
+        throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+      }
+      LOG.debug("Waiting on " + purpose);
+    } while (EnvironmentEdgeManager.currentTime() < done && env.isRunning());
+
+    throw new TimeoutIOException("Timed out while waiting on " + purpose);
+  }
+
+  protected static void waitMetaRegions(final MasterProcedureEnv env) throws IOException {
+    int timeout = env.getMasterConfiguration().getInt("hbase.client.catalog.timeout", 10000);
+    try {
+      if (env.getMasterServices().getMetaTableLocator().waitMetaRegionLocation(
+            env.getMasterServices().getZooKeeper(), timeout) == null) {
+        throw new NotAllMetaRegionsOnlineException();
+      }
+    } catch (InterruptedException e) {
+      throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+    }
+  }
+
+  protected static void waitRegionServers(final MasterProcedureEnv env) throws IOException {
+    final ServerManager sm = env.getMasterServices().getServerManager();
+    ProcedureSyncWait.waitFor(env, "server to assign region(s)",
+        new ProcedureSyncWait.Predicate<Boolean>() {
+      @Override
+      public Boolean evaluate() throws IOException {
+        List<ServerName> servers = sm.createDestinationServersList();
+        return servers != null && !servers.isEmpty();
+      }
+    });
+  }
+
+  protected static List<HRegionInfo> getRegionsFromMeta(final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    return ProcedureSyncWait.waitFor(env, "regions of table=" + tableName + " from meta",
+        new ProcedureSyncWait.Predicate<List<HRegionInfo>>() {
+      @Override
+      public List<HRegionInfo> evaluate() throws IOException {
+        if (TableName.META_TABLE_NAME.equals(tableName)) {
+          return new MetaTableLocator().getMetaRegions(env.getMasterServices().getZooKeeper());
+        }
+        return MetaTableAccessor.getTableRegions(env.getMasterServices().getConnection(),tableName);
+      }
+    });
+  }
+
+  protected static void waitRegionInTransition(final MasterProcedureEnv env,
+      final List<HRegionInfo> regions) throws IOException, CoordinatedStateException {
+    final AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    final RegionStates states = am.getRegionStates();
+    for (final HRegionInfo region : regions) {
+      ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
+          new ProcedureSyncWait.Predicate<Boolean>() {
+        @Override
+        public Boolean evaluate() throws IOException {
+          if (states.isRegionInState(region, State.FAILED_OPEN)) {
+            am.regionOffline(region);
+          }
+          return !states.isRegionInTransition(region);
+        }
+      });
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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
new file mode 100644
index 0000000..76ca094
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TableProcedureInterface.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import org.apache.hadoop.hbase.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, ...)
+ * must implement this interface to allow the system handle the lock/concurrency problems.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface TableProcedureInterface {
+  public enum TableOperationType { CREATE, DELETE, EDIT, READ };
+
+  /**
+   * @return the name of the table the procedure is operating on
+   */
+  TableName getTableName();
+
+  /**
+   * Given an operation type we can take decisions about what to do with pending operations.
+   * e.g. if we get a delete and we have some table operation pending (e.g. add column)
+   * we can abort those operations.
+   * @return the operation type that the procedure is executing.
+   */
+  TableOperationType getTableOperationType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
index 9893fc8..5fe5f8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/MasterQuotaManager.java
@@ -31,7 +31,7 @@ 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.master.MasterServices;
-import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
+import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
@@ -444,14 +444,11 @@ public class MasterQuotaManager implements RegionStateListener {
       new HRegionInfo(QuotaUtil.QUOTA_TABLE_NAME)
     };
 
-    masterServices.getExecutorService()
-      .submit(new CreateTableHandler(masterServices,
-        masterServices.getMasterFileSystem(),
-        QuotaUtil.QUOTA_TABLE_DESC,
-        masterServices.getConfiguration(),
-        newRegions,
-        masterServices)
-          .prepare());
+    masterServices.getMasterProcedureExecutor()
+      .submitProcedure(new CreateTableProcedure(
+          masterServices.getMasterProcedureExecutor().getEnvironment(),
+          QuotaUtil.QUOTA_TABLE_DESC,
+          newRegions));
   }
 
   private static class NamedLock<T> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index a515f8e..f15eb1b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -981,13 +981,14 @@ public class HRegionServer extends HasThread implements
 
     // Send interrupts to wake up threads if sleeping so they notice shutdown.
     // TODO: Should we check they are alive? If OOME could have exited already
-    if(this.hMemManager != null) this.hMemManager.stop();
+    if (this.hMemManager != null) this.hMemManager.stop();
     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
     if (this.compactionChecker != null) this.compactionChecker.cancel(true);
     if (this.healthCheckChore != null) this.healthCheckChore.cancel(true);
     if (this.nonceManagerChore != null) this.nonceManagerChore.cancel(true);
     if (this.storefileRefresher != null) this.storefileRefresher.cancel(true);
+    sendShutdownInterrupt();
 
     // Stop the quota manager
     if (rsQuotaManager != null) {
@@ -2073,6 +2074,12 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
+   * Called on stop/abort before closing the cluster connection and meta locator.
+   */
+  protected void sendShutdownInterrupt() {
+  }
+
+  /**
    * Wait on all threads to finish. Presumption is that all closes and stops
    * have already been called.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 95d8a17..347cad5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -67,6 +67,30 @@ public abstract class ModifyRegionUtils {
     void editRegion(final HRegionInfo region) throws IOException;
   }
 
+  public static HRegionInfo[] createHRegionInfos(HTableDescriptor hTableDescriptor,
+      byte[][] splitKeys) {
+    long regionId = System.currentTimeMillis();
+    HRegionInfo[] hRegionInfos = null;
+    if (splitKeys == null || splitKeys.length == 0) {
+      hRegionInfos = new HRegionInfo[]{
+        new HRegionInfo(hTableDescriptor.getTableName(), null, null, false, regionId)
+      };
+    } else {
+      int numRegions = splitKeys.length + 1;
+      hRegionInfos = new HRegionInfo[numRegions];
+      byte[] startKey = null;
+      byte[] endKey = null;
+      for (int i = 0; i < numRegions; i++) {
+        endKey = (i == splitKeys.length) ? null : splitKeys[i];
+        hRegionInfos[i] =
+             new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
+                 false, regionId);
+        startKey = endKey;
+      }
+    }
+    return hRegionInfos;
+  }
+
   /**
    * Create new set of regions on the specified file-system.
    * NOTE: that you should add the regions to hbase:meta after this operation.

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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 8ed49ff..2c13f39 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
@@ -63,6 +63,8 @@ import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLog
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@@ -261,6 +263,11 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+      return null;
+    }
+
+    @Override
     public ServerManager getServerManager() {
       return null;
     }
@@ -912,7 +919,7 @@ public class TestCatalogJanitor {
     MasterServices services = new MockMasterServices(server);
 
     // create the janitor
-    
+
     CatalogJanitor janitor = new CatalogJanitor(server, services);
 
     // Create regions.

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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
new file mode 100644
index 0000000..d6c19e1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java
@@ -0,0 +1,317 @@
+/**
+ * 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.concurrent.atomic.AtomicInteger;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+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.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class MasterProcedureTestingUtility {
+  private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
+
+  private MasterProcedureTestingUtility() {
+  }
+
+  public static HTableDescriptor createHTD(final TableName tableName, final String... family) {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
+    for (int i = 0; i < family.length; ++i) {
+      htd.addFamily(new HColumnDescriptor(family[i]));
+    }
+    return htd;
+  }
+
+  public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
+      final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
+    HTableDescriptor htd = createHTD(tableName, family);
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
+    return regions;
+  }
+
+  public static void validateTableCreation(final HMaster master, final TableName tableName,
+      final HRegionInfo[] regions, String... family) throws IOException {
+    validateTableCreation(master, tableName, regions, true, family);
+  }
+
+  public static void validateTableCreation(final HMaster master, final TableName tableName,
+      final HRegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
+    // check filesystem
+    final FileSystem fs = master.getMasterFileSystem().getFileSystem();
+    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    assertTrue(fs.exists(tableDir));
+    List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
+    for (int i = 0; i < regions.length; ++i) {
+      Path regionDir = new Path(tableDir, regions[i].getEncodedName());
+      assertTrue(regions[i] + " region dir does not exist", fs.exists(regionDir));
+      assertTrue(allRegionDirs.remove(regionDir));
+      List<Path> allFamilyDirs = FSUtils.getFamilyDirs(fs, regionDir);
+      for (int j = 0; j < family.length; ++j) {
+        final Path familyDir = new Path(regionDir, family[j]);
+        if (hasFamilyDirs) {
+          assertTrue(family[j] + " family dir does not exist", fs.exists(familyDir));
+          assertTrue(allFamilyDirs.remove(familyDir));
+        } else {
+          // TODO: WARN: Modify Table/Families does not create a family dir
+          if (!fs.exists(familyDir)) {
+            LOG.warn(family[j] + " family dir does not exist");
+          }
+          allFamilyDirs.remove(familyDir);
+        }
+      }
+      assertTrue("found extraneous families: " + allFamilyDirs, allFamilyDirs.isEmpty());
+    }
+    assertTrue("found extraneous regions: " + allRegionDirs, allRegionDirs.isEmpty());
+
+    // check meta
+    assertTrue(MetaTableAccessor.tableExists(master.getConnection(), tableName));
+    assertEquals(regions.length, countMetaRegions(master, tableName));
+
+    // check htd
+    TableDescriptor tableDesc = master.getTableDescriptors().getDescriptor(tableName);
+    assertTrue("table descriptor not found", tableDesc != null);
+    HTableDescriptor htd = tableDesc.getHTableDescriptor();
+    assertTrue("table descriptor not found", htd != null);
+    for (int i = 0; i < family.length; ++i) {
+      assertTrue("family not found " + family[i], htd.getFamily(Bytes.toBytes(family[i])) != null);
+    }
+    assertEquals(family.length, htd.getFamilies().size());
+  }
+
+  public static void validateTableDeletion(final HMaster master, final TableName tableName,
+      final HRegionInfo[] regions, String... family) throws IOException {
+    // check filesystem
+    final FileSystem fs = master.getMasterFileSystem().getFileSystem();
+    final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
+    assertFalse(fs.exists(tableDir));
+
+    // check meta
+    assertFalse(MetaTableAccessor.tableExists(master.getConnection(), tableName));
+    assertEquals(0, countMetaRegions(master, tableName));
+
+    // check htd
+    assertTrue("found htd of deleted table",
+      master.getTableDescriptors().getDescriptor(tableName) == null);
+  }
+
+  private static int countMetaRegions(final HMaster master, final TableName tableName)
+      throws IOException {
+    final AtomicInteger actualRegCount = new AtomicInteger(0);
+    final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
+      @Override
+      public boolean visit(Result rowResult) throws IOException {
+        RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
+        if (list == null) {
+          LOG.warn("No serialized HRegionInfo in " + rowResult);
+          return true;
+        }
+        HRegionLocation l = list.getRegionLocation();
+        if (l == null) {
+          return true;
+        }
+        if (!l.getRegionInfo().getTable().equals(tableName)) {
+          return false;
+        }
+        if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
+        HRegionLocation[] locations = list.getRegionLocations();
+        for (HRegionLocation location : locations) {
+          if (location == null) continue;
+          ServerName serverName = location.getServerName();
+          // Make sure that regions are assigned to server
+          if (serverName != null && serverName.getHostAndPort() != null) {
+            actualRegCount.incrementAndGet();
+          }
+        }
+        return true;
+      }
+    };
+    MetaTableAccessor.scanMetaForTableRegions(master.getConnection(), visitor, tableName);
+    return actualRegCount.get();
+  }
+
+  public static <TState> void testRecoveryAndDoubleExecution(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
+      final int numSteps, final TState[] states) throws Exception {
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    assertEquals(false, procExec.isRunning());
+    // Restart the executor and execute the step twice
+    //   execute step N - kill before store update
+    //   restart executor/store
+    //   execute step N - save on store
+    for (int i = 0; i < numSteps; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+    assertEquals(true, procExec.isRunning());
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+  }
+
+  public static <TState> void testRollbackAndDoubleExecution(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
+      final int lastStep, final TState[] states) throws Exception {
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+    // Restart the executor and execute the step twice
+    //   execute step N - kill before store update
+    //   restart executor/store
+    //   execute step N - save on store
+    for (int i = 0; i < lastStep; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+
+    // Restart the executor and rollback the step twice
+    //   rollback step N - kill before store update
+    //   restart executor/store
+    //   rollback step N - save on store
+    MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
+      new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
+    procExec.registerListener(abortListener);
+    try {
+      for (int i = lastStep + 1; i >= 0; --i) {
+        LOG.info("Restart " + i +" rollback state: "+ states[i]);
+        ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+        ProcedureTestingUtility.restart(procExec);
+        ProcedureTestingUtility.waitProcedure(procExec, procId);
+      }
+    } finally {
+      assertTrue(procExec.unregisterListener(abortListener));
+    }
+
+    ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
+  }
+
+  public static <TState> void testRollbackAndDoubleExecutionAfterPONR(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
+      final int lastStep, final TState[] states) throws Exception {
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+    // Restart the executor and execute the step twice
+    //   execute step N - kill before store update
+    //   restart executor/store
+    //   execute step N - save on store
+    for (int i = 0; i < lastStep; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+
+    // try to inject the abort
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
+      new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
+    procExec.registerListener(abortListener);
+    try {
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      LOG.info("Restart and execute");
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    } finally {
+      assertTrue(procExec.unregisterListener(abortListener));
+    }
+
+    assertEquals(true, procExec.isRunning());
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+  }
+
+  public static <TState> void testRollbackRetriableFailure(
+      final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
+      final int lastStep, final TState[] states) throws Exception {
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+    // Restart the executor and execute the step twice
+    //   execute step N - kill before store update
+    //   restart executor/store
+    //   execute step N - save on store
+    for (int i = 0; i < lastStep; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+
+    // execute the rollback
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
+      new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
+    procExec.registerListener(abortListener);
+    try {
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      LOG.info("Restart and rollback");
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    } finally {
+      assertTrue(procExec.unregisterListener(abortListener));
+    }
+
+    ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
+  }
+
+  public static class InjectAbortOnLoadListener
+      implements ProcedureExecutor.ProcedureExecutorListener {
+    private final ProcedureExecutor<MasterProcedureEnv> procExec;
+
+    public InjectAbortOnLoadListener(final ProcedureExecutor<MasterProcedureEnv> procExec) {
+      this.procExec = procExec;
+    }
+
+    @Override
+    public void procedureLoaded(long procId) {
+      procExec.abort(procId);
+    }
+
+    @Override
+    public void procedureAdded(long procId) { /* no-op */ }
+
+    @Override
+    public void procedureFinished(long procId) { /* no-op */ }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
new file mode 100644
index 0000000..7cd64b6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+
+import org.apache.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.TableExistsException;
+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.CreateTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+
+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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestCreateTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure.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 {
+    resetProcExecutorTestingKillFlag();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    resetProcExecutorTestingKillFlag();
+    for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
+      LOG.info("Tear down, remove table=" + htd.getTableName());
+      UTIL.deleteTable(htd.getTableName());
+    }
+  }
+
+  private void resetProcExecutorTestingKillFlag() {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleCreate() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleCreate");
+    final byte[][] splitKeys = null;
+    testSimpleCreate(tableName, splitKeys);
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleCreateWithSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleCreateWithSplits");
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    testSimpleCreate(tableName, splitKeys);
+  }
+
+  private void testSimpleCreate(final TableName tableName, byte[][] splitKeys) throws Exception {
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  @Test(timeout=60000, expected=TableExistsException.class)
+  public void testCreateExisting() throws Exception {
+    final TableName tableName = TableName.valueOf("testCreateExisting");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
+    final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+
+    // create the table
+    long procId1 = procExec.submitProcedure(
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+
+    // create another with the same name
+    ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch();
+    long procId2 = procExec.submitProcedure(
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2));
+
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
+
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+    latch2.await();
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+
+    // create the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Create procedure && kill the executor
+    byte[][] splitKeys = null;
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = procExec.submitProcedure(
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+
+    // Restart the executor and execute the step twice
+    // NOTE: the 6 (number of CreateTableState steps) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec, procId, 6, CreateTableState.values());
+
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  @Test(timeout=90000)
+  public void testRollbackAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
+
+    // create the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Create procedure && kill the executor
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+    htd.setRegionReplication(3);
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = procExec.submitProcedure(
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+
+    // NOTE: the 4 (number of CreateTableState steps) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
+        procExec, procId, 4, CreateTableState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+
+    // are we able to create the table after a rollback?
+    resetProcExecutorTestingKillFlag();
+    testSimpleCreate(tableName, splitKeys);
+  }
+
+  @Test(timeout=90000)
+  public void testRollbackRetriableFailure() throws Exception {
+    final TableName tableName = TableName.valueOf("testRollbackRetriableFailure");
+
+    // create the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Create procedure && kill the executor
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = procExec.submitProcedure(
+      new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions));
+
+    // NOTE: the 4 (number of CreateTableState steps) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    MasterProcedureTestingUtility.testRollbackRetriableFailure(
+        procExec, procId, 4, CreateTableState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+
+    // are we able to create the table after a rollback?
+    resetProcExecutorTestingKillFlag();
+    testSimpleCreate(tableName, splitKeys);
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+
+  public static class FaultyCreateTableProcedure extends CreateTableProcedure {
+    private int retries = 0;
+
+    public FaultyCreateTableProcedure() {
+      // Required by the Procedure framework to create the procedure on replay
+    }
+
+    public FaultyCreateTableProcedure(final MasterProcedureEnv env,
+        final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions)
+        throws IOException {
+      super(env, hTableDescriptor, newRegions);
+    }
+
+    @Override
+    protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
+        throws IOException {
+      if (retries++ < 3) {
+        LOG.info("inject rollback failure state=" + state);
+        throw new IOException("injected failure number " + retries);
+      } else {
+        super.rollbackState(env, state);
+        retries = 0;
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
new file mode 100644
index 0000000..6795b22
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteTableProcedure.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import org.apache.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.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+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.DeleteTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestDeleteTableProcedure {
+  private static final Log LOG = LogFactory.getLog(TestDeleteTableProcedure.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 {
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+    assertTrue("expected executor to be running", procExec.isRunning());
+  }
+
+  @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, expected=TableNotFoundException.class)
+  public void testDeleteNotExistentTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteNotExistentTable");
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedurePrepareLatch latch = new ProcedurePrepareLatch.CompatibilityLatch();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+        new DeleteTableProcedure(procExec.getEnvironment(), tableName, latch));
+    latch.await();
+  }
+
+  @Test(timeout=60000, expected=TableNotDisabledException.class)
+  public void testDeleteNotDisabledTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteNotDisabledTable");
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
+
+    ProcedurePrepareLatch latch = new ProcedurePrepareLatch.CompatibilityLatch();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+        new DeleteTableProcedure(procExec.getEnvironment(), tableName, latch));
+    latch.await();
+  }
+
+  @Test(timeout=60000)
+  public void testDeleteDeletedTable() throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteDeletedTable");
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      procExec, tableName, null, "f");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // delete the table (that exists)
+    long procId1 = procExec.submitProcedure(
+        new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+    // delete the table (that will no longer exist)
+    long procId2 = procExec.submitProcedure(
+        new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+
+    // Wait the completion
+    ProcedureTestingUtility.waitProcedure(procExec, procId1);
+    ProcedureTestingUtility.waitProcedure(procExec, procId2);
+
+    // First delete should succeed
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
+
+    // Second delete should fail with TableNotFound
+    ProcedureResult result = procExec.getResult(procId2);
+    assertTrue(result.isFailed());
+    LOG.debug("Delete failed with exception: " + result.getException());
+    assertTrue(result.getException().getCause() instanceof TableNotFoundException);
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleDelete() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleDelete");
+    final byte[][] splitKeys = null;
+    testSimpleDelete(tableName, splitKeys);
+  }
+
+  @Test(timeout=60000)
+  public void testSimpleDeleteWithSplits() throws Exception {
+    final TableName tableName = TableName.valueOf("testSimpleDeleteWithSplits");
+    final byte[][] splitKeys = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+    };
+    testSimpleDelete(tableName, splitKeys);
+  }
+
+  private void testSimpleDelete(final TableName tableName, byte[][] splitKeys) throws Exception {
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    // delete the table
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    long procId = ProcedureTestingUtility.submitAndWait(procExec,
+      new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  @Test(timeout=60000)
+  public void testRecoveryAndDoubleExecution() throws Exception {
+    final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
+
+    // create the table
+    byte[][] splitKeys = null;
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitNoProcedureRunning(procExec);
+    ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+
+    // Restart the executor and execute the step twice
+    // NOTE: the 6 (number of DeleteTableState steps) is hardcoded,
+    //       so you have to look at this test at least once when you add a new step.
+    MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
+      procExec, procId, 6, DeleteTableState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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
new file mode 100644
index 0000000..faf7845
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -0,0 +1,291 @@
+/**
+ * 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.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+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.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestMasterFailoverWithProcedures {
+  private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static void setupConf(Configuration conf) {
+  }
+
+  @Before
+  public void setup() throws Exception {
+    setupConf(UTIL.getConfiguration());
+    UTIL.startMiniCluster(2, 1);
+
+    final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, false);
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, false);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testWalRecoverLease() throws Exception {
+    final ProcedureStore masterStore = getMasterProcedureExecutor().getStore();
+    assertTrue("expected WALStore for this test", masterStore instanceof WALProcedureStore);
+
+    HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
+    // Abort Latch for the master store
+    final CountDownLatch masterStoreAbort = new CountDownLatch(1);
+    masterStore.registerListener(new ProcedureStore.ProcedureStoreListener() {
+      @Override
+      public void abortProcess() {
+        LOG.debug("Abort store of Master");
+        masterStoreAbort.countDown();
+      }
+    });
+
+    // startup a fake master the new WAL store will take the lease
+    // and the active master should abort.
+    HMaster backupMaster3 = Mockito.mock(HMaster.class);
+    Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
+    Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
+    final WALProcedureStore backupStore3 = new WALProcedureStore(firstMaster.getConfiguration(),
+        firstMaster.getMasterFileSystem().getFileSystem(),
+        ((WALProcedureStore)masterStore).getLogDir(),
+        new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
+    // Abort Latch for the test store
+    final CountDownLatch backupStore3Abort = new CountDownLatch(1);
+    backupStore3.registerListener(new ProcedureStore.ProcedureStoreListener() {
+      @Override
+      public void abortProcess() {
+        LOG.debug("Abort store of backupMaster3");
+        backupStore3Abort.countDown();
+        backupStore3.stop(true);
+      }
+    });
+    backupStore3.start(1);
+    backupStore3.recoverLease();
+
+    // Try to trigger a command on the master (WAL lease expired on the active one)
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf("mtb"), "f");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+    LOG.debug("submit proc");
+    getMasterProcedureExecutor().submitProcedure(
+      new CreateTableProcedure(getMasterProcedureExecutor().getEnvironment(), htd, regions));
+    LOG.debug("wait master store abort");
+    masterStoreAbort.await();
+
+    // Now the real backup master should start up
+    LOG.debug("wait backup master to startup");
+    waitBackupMaster(UTIL, firstMaster);
+    assertEquals(true, firstMaster.isStopped());
+
+    // wait the store in here to abort (the test will fail due to timeout if it doesn't)
+    LOG.debug("wait the store to abort");
+    backupStore3.getStoreTracker().setDeleted(1, false);
+    backupStore3.delete(1);
+    backupStore3Abort.await();
+  }
+
+  // ==========================================================================
+  //  Test Create Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testCreateWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestCreateTableProcedure
+    // 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)
+    testCreateWithFailoverAtStep(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS.ordinal());
+  }
+
+  private void testCreateWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testCreateWithFailoverAtStep" + step);
+
+    // create the table
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+    // Start the Create procedure && kill the executor
+    byte[][] splitKeys = null;
+    HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+    HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+    long procId = procExec.submitProcedure(
+      new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
+
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  // ==========================================================================
+  //  Test Delete Table
+  // ==========================================================================
+  @Test(timeout=60000)
+  public void testDeleteWithFailover() throws Exception {
+    // TODO: Should we try every step? (master failover takes long time)
+    // It is already covered by TestDeleteTableProcedure
+    // 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)
+    testDeleteWithFailoverAtStep(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS.ordinal());
+  }
+
+  private void testDeleteWithFailoverAtStep(final int step) throws Exception {
+    final TableName tableName = TableName.valueOf("testDeleteWithFailoverAtStep" + step);
+
+    // create the table
+    byte[][] splitKeys = null;
+    HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+      getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+    Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
+    MasterProcedureTestingUtility.validateTableCreation(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+    UTIL.getHBaseAdmin().disableTable(tableName);
+
+    ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+    // Start the Delete procedure && kill the executor
+    long procId = procExec.submitProcedure(
+      new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+    testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
+
+    MasterProcedureTestingUtility.validateTableDeletion(
+      UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+  }
+
+  // ==========================================================================
+  //  Test Helpers
+  // ==========================================================================
+  public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
+      final long procId, final int lastStepBeforeFailover, TState[] states) throws Exception {
+    ProcedureExecutor<MasterProcedureEnv> procExec =
+      testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+    for (int i = 0; i < lastStepBeforeFailover; ++i) {
+      LOG.info("Restart "+ i +" exec state: " + states[i]);
+      ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+      ProcedureTestingUtility.restart(procExec);
+      ProcedureTestingUtility.waitProcedure(procExec, procId);
+    }
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+
+    LOG.info("Trigger master failover");
+    masterFailover(testUtil);
+
+    procExec = testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+    ProcedureTestingUtility.waitProcedure(procExec, procId);
+    ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+  }
+
+  // ==========================================================================
+  //  Master failover utils
+  // ==========================================================================
+  public static void masterFailover(final HBaseTestingUtility testUtil)
+      throws Exception {
+    MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+    // Kill the master
+    HMaster oldMaster = cluster.getMaster();
+    cluster.killMaster(cluster.getMaster().getServerName());
+
+    // Wait the secondary
+    waitBackupMaster(testUtil, oldMaster);
+  }
+
+  public static void waitBackupMaster(final HBaseTestingUtility testUtil,
+      final HMaster oldMaster) throws Exception {
+    MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+    HMaster newMaster = cluster.getMaster();
+    while (newMaster == null || newMaster == oldMaster) {
+      Thread.sleep(250);
+      newMaster = cluster.getMaster();
+    }
+
+    while (!(newMaster.isActiveMaster() && newMaster.isInitialized())) {
+      Thread.sleep(250);
+    }
+  }
+
+  // ==========================================================================
+  //  Helpers
+  // ==========================================================================
+  private MasterProcedureEnv getMasterProcedureEnv() {
+    return getMasterProcedureExecutor().getEnvironment();
+  }
+
+  private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+  }
+
+  private FileSystem getFileSystem() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+  }
+
+  private Path getRootDir() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+  }
+
+  private Path getTempDir() {
+    return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getTempDir();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureQueue.java
new file mode 100644
index 0000000..d22930f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureQueue.java
@@ -0,0 +1,433 @@
+/**
+ * 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.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.TableLockManager;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestMasterProcedureQueue {
+  private static final Log LOG = LogFactory.getLog(TestMasterProcedureQueue.class);
+
+  private MasterProcedureQueue queue;
+  private Configuration conf;
+
+  @Before
+  public void setUp() throws IOException {
+    conf = HBaseConfiguration.create();
+    queue = new MasterProcedureQueue(conf, new TableLockManager.NullTableLockManager());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    assertEquals(0, queue.size());
+  }
+
+  /**
+   * Verify simple create/insert/fetch/delete of the table queue.
+   */
+  @Test
+  public void testSimpleTableOpsQueues() throws Exception {
+    final int NUM_TABLES = 10;
+    final int NUM_ITEMS = 10;
+
+    int count = 0;
+    for (int i = 1; i <= NUM_TABLES; ++i) {
+      TableName tableName = TableName.valueOf(String.format("test-%04d", i));
+      // insert items
+      for (int j = 1; j <= NUM_ITEMS; ++j) {
+        queue.addBack(new TestTableProcedure(i * 1000 + j, tableName,
+          TableProcedureInterface.TableOperationType.EDIT));
+        assertEquals(++count, queue.size());
+      }
+    }
+    assertEquals(NUM_TABLES * NUM_ITEMS, queue.size());
+
+    for (int j = 1; j <= NUM_ITEMS; ++j) {
+      for (int i = 1; i <= NUM_TABLES; ++i) {
+        Long procId = queue.poll();
+        assertEquals(--count, queue.size());
+        assertEquals(i * 1000 + j, procId.longValue());
+      }
+    }
+    assertEquals(0, queue.size());
+
+    for (int i = 1; i <= NUM_TABLES; ++i) {
+      TableName tableName = TableName.valueOf(String.format("test-%04d", i));
+      // complete the table deletion
+      assertTrue(queue.markTableAsDeleted(tableName));
+    }
+  }
+
+  /**
+   * Check that the table queue is not deletable until every procedure
+   * in-progress is completed (this is a special case for write-locks).
+   */
+  @Test
+  public void testCreateDeleteTableOperationsWithWriteLock() throws Exception {
+    TableName tableName = TableName.valueOf("testtb");
+
+    queue.addBack(new TestTableProcedure(1, tableName,
+          TableProcedureInterface.TableOperationType.EDIT));
+
+    // table can't be deleted because one item is in the queue
+    assertFalse(queue.markTableAsDeleted(tableName));
+
+    // fetch item and take a lock
+    assertEquals(1, queue.poll().longValue());
+    // take the xlock
+    assertTrue(queue.tryAcquireTableWrite(tableName, "write"));
+    // table can't be deleted because we have the lock
+    assertEquals(0, queue.size());
+    assertFalse(queue.markTableAsDeleted(tableName));
+    // release the xlock
+    queue.releaseTableWrite(tableName);
+    // complete the table deletion
+    assertTrue(queue.markTableAsDeleted(tableName));
+  }
+
+  /**
+   * Check that the table queue is not deletable until every procedure
+   * in-progress is completed (this is a special case for read-locks).
+   */
+  @Test
+  public void testCreateDeleteTableOperationsWithReadLock() throws Exception {
+    final TableName tableName = TableName.valueOf("testtb");
+    final int nitems = 2;
+
+    for (int i = 1; i <= nitems; ++i) {
+      queue.addBack(new TestTableProcedure(i, tableName,
+            TableProcedureInterface.TableOperationType.READ));
+    }
+
+    // table can't be deleted because one item is in the queue
+    assertFalse(queue.markTableAsDeleted(tableName));
+
+    for (int i = 1; i <= nitems; ++i) {
+      // fetch item and take a lock
+      assertEquals(i, queue.poll().longValue());
+      // take the rlock
+      assertTrue(queue.tryAcquireTableRead(tableName, "read " + i));
+      // table can't be deleted because we have locks and/or items in the queue
+      assertFalse(queue.markTableAsDeleted(tableName));
+    }
+
+    for (int i = 1; i <= nitems; ++i) {
+      // table can't be deleted because we have locks
+      assertFalse(queue.markTableAsDeleted(tableName));
+      // release the rlock
+      queue.releaseTableRead(tableName);
+    }
+
+    // there are no items and no lock in the queeu
+    assertEquals(0, queue.size());
+    // complete the table deletion
+    assertTrue(queue.markTableAsDeleted(tableName));
+  }
+
+  /**
+   * Verify the correct logic of RWLocks on the queue
+   */
+  @Test
+  public void testVerifyRwLocks() throws Exception {
+    TableName tableName = TableName.valueOf("testtb");
+    queue.addBack(new TestTableProcedure(1, tableName,
+          TableProcedureInterface.TableOperationType.EDIT));
+    queue.addBack(new TestTableProcedure(2, tableName,
+          TableProcedureInterface.TableOperationType.READ));
+    queue.addBack(new TestTableProcedure(3, tableName,
+          TableProcedureInterface.TableOperationType.EDIT));
+    queue.addBack(new TestTableProcedure(4, tableName,
+          TableProcedureInterface.TableOperationType.READ));
+    queue.addBack(new TestTableProcedure(5, tableName,
+          TableProcedureInterface.TableOperationType.READ));
+
+    // Fetch the 1st item and take the write lock
+    Long procId = queue.poll();
+    assertEquals(1, procId.longValue());
+    assertEquals(true, queue.tryAcquireTableWrite(tableName, "write " + procId));
+
+    // Fetch the 2nd item and verify that the lock can't be acquired
+    assertEquals(null, queue.poll());
+
+    // Release the write lock and acquire the read lock
+    queue.releaseTableWrite(tableName);
+
+    // Fetch the 2nd item and take the read lock
+    procId = queue.poll();
+    assertEquals(2, procId.longValue());
+    assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
+
+    // Fetch the 3rd item and verify that the lock can't be acquired
+    procId = queue.poll();
+    assertEquals(3, procId.longValue());
+    assertEquals(false, queue.tryAcquireTableWrite(tableName, "write " + procId));
+
+    // release the rdlock of item 2 and take the wrlock for the 3d item
+    queue.releaseTableRead(tableName);
+    assertEquals(true, queue.tryAcquireTableWrite(tableName, "write " + procId));
+
+    // Fetch 4th item and verify that the lock can't be acquired
+    assertEquals(null, queue.poll());
+
+    // Release the write lock and acquire the read lock
+    queue.releaseTableWrite(tableName);
+
+    // Fetch the 4th item and take the read lock
+    procId = queue.poll();
+    assertEquals(4, procId.longValue());
+    assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
+
+    // Fetch the 4th item and take the read lock
+    procId = queue.poll();
+    assertEquals(5, procId.longValue());
+    assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
+
+    // Release 4th and 5th read-lock
+    queue.releaseTableRead(tableName);
+    queue.releaseTableRead(tableName);
+
+    // remove table queue
+    assertEquals(0, queue.size());
+    assertTrue("queue should be deleted", queue.markTableAsDeleted(tableName));
+  }
+
+  /**
+   * Verify that "write" operations for a single table are serialized,
+   * but different tables can be executed in parallel.
+   */
+  @Test(timeout=90000)
+  public void testConcurrentWriteOps() throws Exception {
+    final TestTableProcSet procSet = new TestTableProcSet(queue);
+
+    final int NUM_ITEMS = 10;
+    final int NUM_TABLES = 4;
+    final AtomicInteger opsCount = new AtomicInteger(0);
+    for (int i = 0; i < NUM_TABLES; ++i) {
+      TableName tableName = TableName.valueOf(String.format("testtb-%04d", i));
+      for (int j = 1; j < NUM_ITEMS; ++j) {
+        procSet.addBack(new TestTableProcedure(i * 100 + j, tableName,
+          TableProcedureInterface.TableOperationType.EDIT));
+        opsCount.incrementAndGet();
+      }
+    }
+    assertEquals(opsCount.get(), queue.size());
+
+    final Thread[] threads = new Thread[NUM_TABLES * 2];
+    final HashSet<TableName> concurrentTables = new HashSet<TableName>();
+    final ArrayList<String> failures = new ArrayList<String>();
+    final AtomicInteger concurrentCount = new AtomicInteger(0);
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          while (opsCount.get() > 0) {
+            try {
+              TableProcedureInterface proc = procSet.acquire();
+              if (proc == null) {
+                queue.signalAll();
+                if (opsCount.get() > 0) {
+                  continue;
+                }
+                break;
+              }
+              synchronized (concurrentTables) {
+                assertTrue("unexpected concurrency on " + proc.getTableName(),
+                  concurrentTables.add(proc.getTableName()));
+              }
+              assertTrue(opsCount.decrementAndGet() >= 0);
+              try {
+                long procId = ((Procedure)proc).getProcId();
+                TableName tableId = proc.getTableName();
+                int concurrent = concurrentCount.incrementAndGet();
+                assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_TABLES,
+                  concurrent >= 1 && concurrent <= NUM_TABLES);
+                LOG.debug("[S] tableId="+ tableId +" procId="+ procId +" concurrent="+ concurrent);
+                Thread.sleep(2000);
+                concurrent = concurrentCount.decrementAndGet();
+                LOG.debug("[E] tableId="+ tableId +" procId="+ procId +" concurrent="+ concurrent);
+                assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_TABLES);
+              } finally {
+                synchronized (concurrentTables) {
+                  assertTrue(concurrentTables.remove(proc.getTableName()));
+                }
+                procSet.release(proc);
+              }
+            } catch (Throwable e) {
+              LOG.error("Failed " + e.getMessage(), e);
+              synchronized (failures) {
+                failures.add(e.getMessage());
+              }
+            } finally {
+              queue.signalAll();
+            }
+          }
+        }
+      };
+      threads[i].start();
+    }
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].join();
+    }
+    assertTrue(failures.toString(), failures.isEmpty());
+    assertEquals(0, opsCount.get());
+    assertEquals(0, queue.size());
+
+    for (int i = 1; i <= NUM_TABLES; ++i) {
+      TableName table = TableName.valueOf(String.format("testtb-%04d", i));
+      assertTrue("queue should be deleted, table=" + table, queue.markTableAsDeleted(table));
+    }
+  }
+
+  public static class TestTableProcSet {
+    private final MasterProcedureQueue queue;
+    private Map<Long, TableProcedureInterface> procsMap =
+      new ConcurrentHashMap<Long, TableProcedureInterface>();
+
+    public TestTableProcSet(final MasterProcedureQueue queue) {
+      this.queue = queue;
+    }
+
+    public void addBack(TableProcedureInterface tableProc) {
+      Procedure proc = (Procedure)tableProc;
+      procsMap.put(proc.getProcId(), tableProc);
+      queue.addBack(proc);
+    }
+
+    public void addFront(TableProcedureInterface tableProc) {
+      Procedure proc = (Procedure)tableProc;
+      procsMap.put(proc.getProcId(), tableProc);
+      queue.addFront(proc);
+    }
+
+    public TableProcedureInterface acquire() {
+      TableProcedureInterface proc = null;
+      boolean avail = false;
+      while (!avail) {
+        Long procId = queue.poll();
+        proc = procId != null ? procsMap.remove(procId) : null;
+        if (proc == null) break;
+        switch (proc.getTableOperationType()) {
+          case CREATE:
+          case DELETE:
+          case EDIT:
+            avail = queue.tryAcquireTableWrite(proc.getTableName(),
+              "op="+ proc.getTableOperationType());
+            break;
+          case READ:
+            avail = queue.tryAcquireTableRead(proc.getTableName(),
+              "op="+ proc.getTableOperationType());
+            break;
+        }
+        if (!avail) {
+          addFront(proc);
+          LOG.debug("yield procId=" + procId);
+        }
+      }
+      return proc;
+    }
+
+    public void release(TableProcedureInterface proc) {
+      switch (proc.getTableOperationType()) {
+        case CREATE:
+        case DELETE:
+        case EDIT:
+          queue.releaseTableWrite(proc.getTableName());
+          break;
+        case READ:
+          queue.releaseTableRead(proc.getTableName());
+          break;
+      }
+    }
+  }
+
+  public static class TestTableProcedure extends Procedure<Void>
+      implements TableProcedureInterface {
+    private final TableOperationType opType;
+    private final TableName tableName;
+
+    public TestTableProcedure() {
+      throw new UnsupportedOperationException("recovery should not be triggered here");
+    }
+
+    public TestTableProcedure(long procId, TableName tableName, TableOperationType opType) {
+      this.tableName = tableName;
+      this.opType = opType;
+      setProcId(procId);
+    }
+
+    @Override
+    public TableName getTableName() {
+      return tableName;
+    }
+
+    @Override
+    public TableOperationType getTableOperationType() {
+      return opType;
+    }
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      return null;
+    }
+
+    @Override
+    protected void rollback(Void env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected boolean abort(Void env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected void serializeStateData(final OutputStream stream) throws IOException {}
+
+    @Override
+    protected void deserializeStateData(final InputStream stream) throws IOException {}
+  }
+}


[06/18] 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/711ae17a/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..5c96f21 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 ModifyTableMessageOrBuilder
+      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 ModifyTableMessage}
+   */
+  public static final class ModifyTableMessage extends
+      com.google.protobuf.GeneratedMessage
+      implements ModifyTableMessageOrBuilder {
+    // Use ModifyTableMessage.newBuilder() to construct.
+    private ModifyTableMessage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ModifyTableMessage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ModifyTableMessage defaultInstance;
+    public static ModifyTableMessage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ModifyTableMessage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ModifyTableMessage(
+        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_ModifyTableMessage_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableMessage_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ModifyTableMessage> PARSER =
+        new com.google.protobuf.AbstractParser<ModifyTableMessage>() {
+      public ModifyTableMessage parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ModifyTableMessage(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ModifyTableMessage> 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.ModifyTableMessage)) {
         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.ModifyTableMessage other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage) 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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.ModifyTableMessage 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 ModifyTableMessage}
      */
     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.ModifyTableMessageOrBuilder {
       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_ModifyTableMessage_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_ModifyTableMessage_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.ModifyTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.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_ModifyTableMessage_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.ModifyTableMessage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.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.ModifyTableMessage build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage 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.ModifyTableMessage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage(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.ModifyTableMessage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage)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.ModifyTableMessage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableMessage.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.ModifyTableMessage 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.ModifyTableMessage) 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.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.gene

<TRUNCATED>

[14/18] 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/5946a525
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5946a525
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5946a525

Branch: refs/heads/hbase-12439
Commit: 5946a5255aa067a4180850a7b1d148e0c6293471
Parents: 711ae17
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 22:45:00 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 5169 +++++++++++++++++-
 .../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(+), 116 deletions(-)
----------------------------------------------------------------------



[17/18] 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/6fbf41dd
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6fbf41dd
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6fbf41dd

Branch: refs/heads/hbase-12439
Commit: 6fbf41dd7a2be4ce3ed60c5c5a2407b77c0d6b83
Parents: 5946a52
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 22:45:00 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 2254 +++++++++++++++++-
 .../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, 3802 insertions(+), 122 deletions(-)
----------------------------------------------------------------------



[02/18] hbase git commit: HBASE-13203 Procedure v2 - master create/delete table

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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
new file mode 100644
index 0000000..4e9b05e
--- /dev/null
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -0,0 +1,74 @@
+/**
+ * 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.
+ */
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "MasterProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "HBase.proto";
+import "RPC.proto";
+
+// ============================================================================
+//  WARNING - Compatibility rules
+// ============================================================================
+// This .proto contains the data serialized by the master procedures.
+// Each procedure has some state stored to know, which step were executed
+// and what were the parameters or data created by the previous steps.
+// new code should be able to handle the old format or at least fail cleanly
+// triggering a rollback/cleanup.
+//
+// Procedures that are inheriting from a StateMachineProcedure have an enum:
+//  - Do not change the number of the 'State' enums.
+//    doing so, will cause executing the wrong 'step' on the pending
+//    procedures when they will be replayed.
+//  - Do not remove items from the enum, new code must be able to handle
+//    all the previous 'steps'. There may be pending procedure ready to be
+//    recovered replayed. alternative you can make sure that not-known state
+//    will result in a failure that will rollback the already executed steps.
+// ============================================================================
+
+enum CreateTableState {
+  CREATE_TABLE_PRE_OPERATION = 1;
+  CREATE_TABLE_WRITE_FS_LAYOUT = 2;
+  CREATE_TABLE_ADD_TO_META = 3;
+  CREATE_TABLE_ASSIGN_REGIONS = 4;
+  CREATE_TABLE_UPDATE_DESC_CACHE = 5;
+  CREATE_TABLE_POST_OPERATION = 6;
+}
+
+message CreateTableStateData {
+  required UserInformation user_info = 1;
+  required TableSchema table_schema = 2;
+  repeated RegionInfo region_info = 3;
+}
+
+enum DeleteTableState {
+  DELETE_TABLE_PRE_OPERATION = 1;
+  DELETE_TABLE_REMOVE_FROM_META = 2;
+  DELETE_TABLE_CLEAR_FS_LAYOUT = 3;
+  DELETE_TABLE_UPDATE_DESC_CACHE = 4;
+  DELETE_TABLE_UNASSIGN_REGIONS = 5;
+  DELETE_TABLE_POST_OPERATION = 6;
+}
+
+message DeleteTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  repeated RegionInfo region_info = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index edebb1a..107480a 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -316,6 +316,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-procedure</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
     </dependency>
     <dependency>
@@ -336,6 +340,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-procedure</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>commons-httpclient</groupId>
       <artifactId>commons-httpclient</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index 971fa50..0da16a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -21,6 +21,7 @@ import java.net.InetAddress;
 
 import org.apache.hadoop.hbase.security.User;
 
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
 
 public interface RpcCallContext extends Delayable {
   /**
@@ -57,4 +58,9 @@ public interface RpcCallContext extends Delayable {
    * @return Address of remote client if a request is ongoing, else null
    */
   InetAddress getRemoteAddress();
+
+  /**
+   * @return the client version info, or null if the information is not present
+   */
+  VersionInfo getClientVersionInfo();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 770f4cd..c69a187 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.AuthMethod;
@@ -399,7 +400,7 @@ public class RpcServer implements RpcServerInterface {
           // Set the exception as the result of the method invocation.
           headerBuilder.setException(exceptionBuilder.build());
         }
-        // Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the 
+        // Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the
         // reservoir when finished. This is hacky and the hack is not contained but benefits are
         // high when we can avoid a big buffer allocation on each rpc.
         this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec,
@@ -544,6 +545,11 @@ public class RpcServer implements RpcServerInterface {
     public InetAddress getRemoteAddress() {
       return remoteAddress;
     }
+
+    @Override
+    public VersionInfo getClientVersionInfo() {
+      return connection.getVersionInfo();
+    }
   }
 
   /** Listens on the socket. Creates jobs for the handler threads*/
@@ -1273,6 +1279,13 @@ public class RpcServer implements RpcServerInterface {
       this.lastContact = lastContact;
     }
 
+    public VersionInfo getVersionInfo() {
+      if (connectionHeader.hasVersionInfo()) {
+        return connectionHeader.getVersionInfo();
+      }
+      return null;
+    }
+
     /* Return true if the connection has no outstanding rpc */
     private boolean isIdle() {
       return rpcCount.get() == 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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 581e3c9..8ec883a 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,8 +90,6 @@ 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.CreateTableHandler;
-import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
@@ -100,11 +98,18 @@ 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.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.ProcedurePrepareLatch;
 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.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;
@@ -123,6 +128,7 @@ import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
@@ -290,6 +296,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   // it is assigned after 'initialized' guard set to true, so should be volatile
   private volatile MasterQuotaManager quotaManager;
 
+  private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
+  private WALProcedureStore procedureStore;
+
   // handle table states
   private TableStateManager tableStateManager;
 
@@ -1002,6 +1011,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
    // Any time changing this maxThreads to > 1, pls see the comment at
    // AccessController#postCreateTableHandler
    this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
+   startProcedureExecutor();
 
    // Start log cleaner thread
    int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
@@ -1023,6 +1033,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
+  protected void sendShutdownInterrupt() {
+    super.sendShutdownInterrupt();
+    stopProcedureExecutor();
+  }
+
+  @Override
   protected void stopServiceThreads() {
     if (masterJettyServer != null) {
       LOG.info("Stopping master jetty server");
@@ -1034,6 +1050,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
     super.stopServiceThreads();
     stopChores();
+
     // Wait for all the remaining region servers to report in IFF we were
     // running a cluster shutdown AND we were NOT aborting.
     if (!isAborted() && this.serverManager != null &&
@@ -1054,6 +1071,34 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
   }
 
+  private void startProcedureExecutor() throws IOException {
+    final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
+    final Path logDir = new Path(fileSystemManager.getRootDir(),
+        MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
+
+    procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
+        new MasterProcedureEnv.WALStoreLeaseRecovery(this));
+    procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
+    procedureExecutor = new ProcedureExecutor(conf, procEnv, procedureStore,
+        procEnv.getProcedureQueue());
+
+    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
+        Math.max(Runtime.getRuntime().availableProcessors(),
+          MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
+    procedureStore.start(numThreads);
+    procedureExecutor.start(numThreads);
+  }
+
+  private void stopProcedureExecutor() {
+    if (procedureExecutor != null) {
+      procedureExecutor.stop();
+    }
+
+    if (procedureStore != null) {
+      procedureStore.stop(isAborted());
+    }
+  }
+
   private void stopChores() {
     if (this.balancerChore != null) {
       this.balancerChore.cancel(true);
@@ -1290,7 +1335,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
     ensureNamespaceExists(namespace);
 
-    HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
+    HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
     sanityCheckTableDescriptor(hTableDescriptor);
     this.quotaManager.checkNamespaceTableAndRegionQuota(hTableDescriptor.getTableName(),
@@ -1299,13 +1344,22 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.preCreateTable(hTableDescriptor, newRegions);
     }
     LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
-    this.service.submit(new CreateTableHandler(this,
-      this.fileSystemManager, hTableDescriptor, conf,
-      newRegions, this).prepare());
+
+    // TODO: We can handle/merge duplicate requests, and differentiate the case of
+    //       TableExistsException by saying if the schema is the same or not.
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
+    long procId = this.procedureExecutor.submitProcedure(
+      new CreateTableProcedure(procedureExecutor.getEnvironment(),
+        hTableDescriptor, newRegions, latch));
+    latch.await();
+
     if (cpHost != null) {
       cpHost.postCreateTable(hTableDescriptor, newRegions);
     }
 
+    // TODO: change the interface to return the procId,
+    //       and add it to the response protobuf.
+    //return procId;
   }
 
   /**
@@ -1512,29 +1566,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
   }
 
-  private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
-    byte[][] splitKeys) {
-    long regionId = System.currentTimeMillis();
-    HRegionInfo[] hRegionInfos = null;
-    if (splitKeys == null || splitKeys.length == 0) {
-      hRegionInfos = new HRegionInfo[]{new HRegionInfo(hTableDescriptor.getTableName(), null, null,
-                false, regionId)};
-    } else {
-      int numRegions = splitKeys.length + 1;
-      hRegionInfos = new HRegionInfo[numRegions];
-      byte[] startKey = null;
-      byte[] endKey = null;
-      for (int i = 0; i < numRegions; i++) {
-        endKey = (i == splitKeys.length) ? null : splitKeys[i];
-        hRegionInfos[i] =
-             new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
-                 false, regionId);
-        startKey = endKey;
-      }
-    }
-    return hRegionInfos;
-  }
-
   private static boolean isCatalogTable(final TableName tableName) {
     return tableName.equals(TableName.META_TABLE_NAME);
   }
@@ -1546,10 +1577,20 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.preDeleteTable(tableName);
     }
     LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
-    this.service.submit(new DeleteTableHandler(tableName, this, this).prepare());
+
+    // TODO: We can handle/merge duplicate request
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
+    long procId = this.procedureExecutor.submitProcedure(
+        new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
+    latch.await();
+
     if (cpHost != null) {
       cpHost.postDeleteTable(tableName);
     }
+
+    // TODO: change the interface to return the procId,
+    //       and add it to the response protobuf.
+    //return procId;
   }
 
   @Override
@@ -1851,6 +1892,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
+  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
+    return procedureExecutor;
+  }
+
+  @Override
   public ServerName getServerName() {
     return this.serverName;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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 63f3119..7352fe8 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
@@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
@@ -82,6 +84,11 @@ public interface MasterServices extends Server {
   MasterQuotaManager getMasterQuotaManager();
 
   /**
+   * @return Master's instance of {@link ProcedureExecutor}
+   */
+  ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor();
+
+  /**
    * Check table is modifiable; i.e. exists and is offline.
    * @param tableName Name of table to check.
    * @throws TableNotDisabledException

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index f0f8fdd..02912b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
+import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -231,18 +231,15 @@ public class TableNamespaceManager {
   }
 
   private void createNamespaceTable(MasterServices masterServices) throws IOException {
-    HRegionInfo newRegions[] = new HRegionInfo[]{
+    HRegionInfo[] newRegions = new HRegionInfo[]{
         new HRegionInfo(HTableDescriptor.NAMESPACE_TABLEDESC.getTableName(), null, null)};
 
-    //we need to create the table this way to bypass
-    //checkInitialized
-    masterServices.getExecutorService()
-        .submit(new CreateTableHandler(masterServices,
-            masterServices.getMasterFileSystem(),
-            HTableDescriptor.NAMESPACE_TABLEDESC,
-            masterServices.getConfiguration(),
-            newRegions,
-            masterServices).prepare());
+    // we need to create the table this way to bypass checkInitialized
+    masterServices.getMasterProcedureExecutor()
+      .submitProcedure(new CreateTableProcedure(
+          masterServices.getMasterProcedureExecutor().getEnvironment(),
+          HTableDescriptor.NAMESPACE_TABLEDESC,
+          newRegions));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
new file mode 100644
index 0000000..dd6d387
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -0,0 +1,442 @@
+/**
+ * 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.InputStream;
+import java.io.OutputStream;
+import java.io.IOException;
+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.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableDescriptor;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+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.CreateTableState;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import com.google.common.collect.Lists;
+
+@InterfaceAudience.Private
+public class CreateTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, CreateTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
+
+  private final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  // used for compatibility with old clients
+  private final ProcedurePrepareLatch syncLatch;
+
+  private HTableDescriptor hTableDescriptor;
+  private List<HRegionInfo> newRegions;
+  private UserGroupInformation user;
+
+  public CreateTableProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    syncLatch = null;
+  }
+
+  public CreateTableProcedure(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions)
+      throws IOException {
+    this(env, hTableDescriptor, newRegions, null);
+  }
+
+  public CreateTableProcedure(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
+      final ProcedurePrepareLatch syncLatch)
+      throws IOException {
+    this.hTableDescriptor = hTableDescriptor;
+    this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
+    this.user = env.getRequestUser().getUGI();
+
+    // used for compatibility with clients without procedures
+    // they need a sync TableExistsException
+    this.syncLatch = syncLatch;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    try {
+      switch (state) {
+        case CREATE_TABLE_PRE_OPERATION:
+          // Verify if we can create the table
+          boolean exists = !prepareCreate(env);
+          ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+
+          if (exists) {
+            assert isFailed() : "the delete should have an exception here";
+            return Flow.NO_MORE_STATE;
+          }
+
+          preCreate(env);
+          setNextState(CreateTableState.CREATE_TABLE_WRITE_FS_LAYOUT);
+          break;
+        case CREATE_TABLE_WRITE_FS_LAYOUT:
+          newRegions = createFsLayout(env, hTableDescriptor, newRegions);
+          setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
+          break;
+        case CREATE_TABLE_ADD_TO_META:
+          newRegions = addTableToMeta(env, hTableDescriptor, newRegions);
+          setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
+          break;
+        case CREATE_TABLE_ASSIGN_REGIONS:
+          assignRegions(env, getTableName(), newRegions);
+          setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
+          break;
+        case CREATE_TABLE_UPDATE_DESC_CACHE:
+          updateTableDescCache(env, getTableName());
+          setNextState(CreateTableState.CREATE_TABLE_POST_OPERATION);
+          break;
+        case CREATE_TABLE_POST_OPERATION:
+          postCreate(env);
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.error("Error trying to create table=" + getTableName() + " state=" + state, e);
+      setFailure("master-create-table", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
+      throws IOException {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+        case CREATE_TABLE_POST_OPERATION:
+          break;
+        case CREATE_TABLE_UPDATE_DESC_CACHE:
+          DeleteTableProcedure.deleteTableDescriptorCache(env, getTableName());
+          break;
+        case CREATE_TABLE_ASSIGN_REGIONS:
+          DeleteTableProcedure.deleteAssignmentState(env, getTableName());
+          break;
+        case CREATE_TABLE_ADD_TO_META:
+          DeleteTableProcedure.deleteFromMeta(env, getTableName(), newRegions);
+          break;
+        case CREATE_TABLE_WRITE_FS_LAYOUT:
+          DeleteTableProcedure.deleteFromFs(env, getTableName(), newRegions, false);
+          break;
+        case CREATE_TABLE_PRE_OPERATION:
+          DeleteTableProcedure.deleteTableStates(env, getTableName());
+          // TODO-MAYBE: call the deleteTable coprocessor event?
+          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 rollback attempt step=" + state + " table=" + getTableName(), e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected CreateTableState getState(final int stateId) {
+    return CreateTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final CreateTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected CreateTableState getInitialState() {
+    return CreateTableState.CREATE_TABLE_PRE_OPERATION;
+  }
+
+  @Override
+  protected void setNextState(final CreateTableState state) {
+    if (aborted.get()) {
+      setAbortFailure("create-table", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public TableName getTableName() {
+    return hTableDescriptor.getTableName();
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.CREATE;
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(getTableName());
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.CreateTableStateData.Builder state =
+      MasterProcedureProtos.CreateTableStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
+        .setTableSchema(hTableDescriptor.convert());
+    if (newRegions != null) {
+      for (HRegionInfo hri: newRegions) {
+        state.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.CreateTableStateData state =
+      MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
+    hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
+    if (state.getRegionInfoCount() == 0) {
+      newRegions = null;
+    } else {
+      newRegions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
+        newRegions.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "create table");
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(getTableName());
+  }
+
+  private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
+    final TableName tableName = getTableName();
+    if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
+      setFailure("master-create-table", new TableExistsException(getTableName()));
+      return false;
+    }
+    return true;
+  }
+
+  private void preCreate(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final HRegionInfo[] regions = newRegions == null ? null :
+        newRegions.toArray(new HRegionInfo[newRegions.size()]);
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.preCreateTableHandler(hTableDescriptor, regions);
+          return null;
+        }
+      });
+    }
+  }
+
+  private void postCreate(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final HRegionInfo[] regions = (newRegions == null) ? null :
+        newRegions.toArray(new HRegionInfo[newRegions.size()]);
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.postCreateTableHandler(hTableDescriptor, regions);
+          return null;
+        }
+      });
+    }
+  }
+
+  protected interface CreateHdfsRegions {
+    List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
+      final Path tableRootDir, final TableName tableName,
+      final List<HRegionInfo> newRegions) throws IOException;
+  }
+
+  protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor, final List<HRegionInfo> newRegions)
+      throws IOException {
+    return createFsLayout(env, hTableDescriptor, newRegions, new CreateHdfsRegions() {
+      @Override
+      public List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
+          final Path tableRootDir, final TableName tableName,
+          final List<HRegionInfo> newRegions) throws IOException {
+        HRegionInfo[] regions = newRegions != null ?
+          newRegions.toArray(new HRegionInfo[newRegions.size()]) : null;
+        return ModifyRegionUtils.createRegions(env.getMasterConfiguration(),
+            tableRootDir, hTableDescriptor, regions, null);
+      }
+    });
+  }
+
+  protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions,
+      final CreateHdfsRegions hdfsRegionHandler) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final Path tempdir = mfs.getTempDir();
+
+    // 1. Create Table Descriptor
+    // using a copy of descriptor, table will be created enabling first
+    TableDescriptor underConstruction = new TableDescriptor(hTableDescriptor);
+    final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
+    ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
+        .createTableDescriptorForTableDirectory(
+          tempTableDir, underConstruction, false);
+
+    // 2. Create Regions
+    newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
+      hTableDescriptor.getTableName(), newRegions);
+
+    // 3. Move Table temp directory to the hbase root location
+    final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), hTableDescriptor.getTableName());
+    FileSystem fs = mfs.getFileSystem();
+    if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
+      throw new IOException("Couldn't delete " + tableDir);
+    }
+    if (!fs.rename(tempTableDir, tableDir)) {
+      throw new IOException("Unable to move table from temp=" + tempTableDir +
+        " to hbase root=" + tableDir);
+    }
+    return newRegions;
+  }
+
+  protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor,
+      final List<HRegionInfo> regions) throws IOException {
+    if (regions != null && regions.size() > 0) {
+      ProcedureSyncWait.waitMetaRegions(env);
+
+      // Add regions to META
+      addRegionsToMeta(env, hTableDescriptor, regions);
+      // Add replicas if needed
+      List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
+
+      // Setup replication for region replicas if needed
+      if (hTableDescriptor.getRegionReplication() > 1) {
+        ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
+      }
+      return newRegions;
+    }
+    return regions;
+  }
+
+  /**
+   * Create any replicas for the regions (the default replicas that was
+   * already created is passed to the method)
+   * @param hTableDescriptor descriptor to use
+   * @param regions default replicas
+   * @return the combined list of default and non-default replicas
+   */
+  private static List<HRegionInfo> addReplicas(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor,
+      final List<HRegionInfo> regions) {
+    int numRegionReplicas = hTableDescriptor.getRegionReplication() - 1;
+    if (numRegionReplicas <= 0) {
+      return regions;
+    }
+    List<HRegionInfo> hRegionInfos =
+        new ArrayList<HRegionInfo>((numRegionReplicas+1)*regions.size());
+    for (int i = 0; i < regions.size(); i++) {
+      for (int j = 1; j <= numRegionReplicas; j++) {
+        hRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(i), j));
+      }
+    }
+    hRegionInfos.addAll(regions);
+    return hRegionInfos;
+  }
+
+  protected static void assignRegions(final MasterProcedureEnv env,
+      final TableName tableName, final List<HRegionInfo> regions) throws IOException {
+    ProcedureSyncWait.waitRegionServers(env);
+
+    // Trigger immediate assignment of the regions in round-robin fashion
+    final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
+    ModifyRegionUtils.assignRegions(assignmentManager, regions);
+
+    // Enable table
+    assignmentManager.getTableStateManager()
+      .setTableState(tableName, TableState.State.ENABLED);
+  }
+
+  /**
+   * Add the specified set of regions to the hbase:meta table.
+   */
+  protected static void addRegionsToMeta(final MasterProcedureEnv env,
+      final HTableDescriptor hTableDescriptor,
+      final List<HRegionInfo> regionInfos) throws IOException {
+    MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(),
+      regionInfos, hTableDescriptor.getRegionReplication());
+  }
+
+  protected static void updateTableDescCache(final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    env.getMasterServices().getTableDescriptors().get(tableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
new file mode 100644
index 0000000..ad5e671
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -0,0 +1,420 @@
+/**
+ * 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.InputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotDisabledException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Delete;
+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.exceptions.HBaseException;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+public class DeleteTableProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, DeleteTableState>
+    implements TableProcedureInterface {
+  private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
+
+  private List<HRegionInfo> regions;
+  private UserGroupInformation user;
+  private TableName tableName;
+
+  // used for compatibility with old clients
+  private final ProcedurePrepareLatch syncLatch;
+
+  public DeleteTableProcedure() {
+    // Required by the Procedure framework to create the procedure on replay
+    syncLatch = null;
+  }
+
+  public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
+    this(env, tableName, null);
+  }
+
+  public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
+      final ProcedurePrepareLatch syncLatch) throws IOException {
+    this.tableName = tableName;
+    this.user = env.getRequestUser().getUGI();
+
+    // used for compatibility with clients without procedures
+    // they need a sync TableNotFoundException, TableNotDisabledException, ...
+    this.syncLatch = syncLatch;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, DeleteTableState state) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+    try {
+      switch (state) {
+        case DELETE_TABLE_PRE_OPERATION:
+          // Verify if we can delete the table
+          boolean deletable = prepareDelete(env);
+          ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+          if (!deletable) {
+            assert isFailed() : "the delete should have an exception here";
+            return Flow.NO_MORE_STATE;
+          }
+
+          preDelete(env);
+
+          // TODO: Move out... in the acquireLock()
+          LOG.debug("waiting for '" + getTableName() + "' regions in transition");
+          regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+          assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
+          ProcedureSyncWait.waitRegionInTransition(env, regions);
+
+          setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
+          break;
+        case DELETE_TABLE_REMOVE_FROM_META:
+          LOG.debug("delete '" + getTableName() + "' regions from META");
+          DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
+          setNextState(DeleteTableState.DELETE_TABLE_CLEAR_FS_LAYOUT);
+          break;
+        case DELETE_TABLE_CLEAR_FS_LAYOUT:
+          LOG.debug("delete '" + getTableName() + "' from filesystem");
+          DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
+          setNextState(DeleteTableState.DELETE_TABLE_UPDATE_DESC_CACHE);
+          break;
+        case DELETE_TABLE_UPDATE_DESC_CACHE:
+          LOG.debug("delete '" + getTableName() + "' descriptor");
+          DeleteTableProcedure.deleteTableDescriptorCache(env, getTableName());
+          setNextState(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS);
+          break;
+        case DELETE_TABLE_UNASSIGN_REGIONS:
+          LOG.debug("delete '" + getTableName() + "' assignment state");
+          DeleteTableProcedure.deleteAssignmentState(env, getTableName());
+          setNextState(DeleteTableState.DELETE_TABLE_POST_OPERATION);
+          break;
+        case DELETE_TABLE_POST_OPERATION:
+          postDelete(env);
+          LOG.debug("delete '" + getTableName() + "' completed");
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (HBaseException|IOException e) {
+      LOG.warn("Retriable error trying to delete table=" + getTableName() + " state=" + state, e);
+    } catch (InterruptedException e) {
+      // if the interrupt is real, the executor will be stopped.
+      LOG.warn("Interrupted trying to delete table=" + getTableName() + " state=" + state, e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final DeleteTableState state) {
+    if (state == DeleteTableState.DELETE_TABLE_PRE_OPERATION) {
+      // nothing to rollback, pre-delete 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 DeleteTableState getState(final int stateId) {
+    return DeleteTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final DeleteTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected DeleteTableState getInitialState() {
+    return DeleteTableState.DELETE_TABLE_PRE_OPERATION;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.DELETE;
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    // TODO: We may be able to abort if the procedure is not started yet.
+    return false;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "delete table");
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(getTableName());
+  }
+
+  @Override
+  public void toStringClassDetails(StringBuilder sb) {
+    sb.append(getClass().getSimpleName());
+    sb.append(" (table=");
+    sb.append(getTableName());
+    sb.append(") user=");
+    sb.append(user);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.DeleteTableStateData.Builder state =
+      MasterProcedureProtos.DeleteTableStateData.newBuilder()
+        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
+        .setTableName(ProtobufUtil.toProtoTableName(tableName));
+    if (regions != null) {
+      for (HRegionInfo hri: regions) {
+        state.addRegionInfo(HRegionInfo.convert(hri));
+      }
+    }
+    state.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.DeleteTableStateData state =
+      MasterProcedureProtos.DeleteTableStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
+    tableName = ProtobufUtil.toTableName(state.getTableName());
+    if (state.getRegionInfoCount() == 0) {
+      regions = null;
+    } else {
+      regions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
+      for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
+        regions.add(HRegionInfo.convert(hri));
+      }
+    }
+  }
+
+  private boolean prepareDelete(final MasterProcedureEnv env) throws IOException {
+    try {
+      env.getMasterServices().checkTableModifiable(tableName);
+    } catch (TableNotFoundException|TableNotDisabledException e) {
+      setFailure("master-delete-table", e);
+      return false;
+    }
+    return true;
+  }
+
+  private boolean preDelete(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final TableName tableName = this.tableName;
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.preDeleteTableHandler(tableName);
+          return null;
+        }
+      });
+    }
+    return true;
+  }
+
+  private void postDelete(final MasterProcedureEnv env)
+      throws IOException, InterruptedException {
+    deleteTableStates(env, tableName);
+
+    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      final TableName tableName = this.tableName;
+      user.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          cpHost.postDeleteTableHandler(tableName);
+          return null;
+        }
+      });
+    }
+  }
+
+  protected static void deleteFromFs(final MasterProcedureEnv env,
+      final TableName tableName, final List<HRegionInfo> regions,
+      final boolean archive) throws IOException {
+    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    final FileSystem fs = mfs.getFileSystem();
+    final Path tempdir = mfs.getTempDir();
+
+    final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
+    final Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
+
+    if (fs.exists(tableDir)) {
+      // Ensure temp exists
+      if (!fs.exists(tempdir) && !fs.mkdirs(tempdir)) {
+        throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
+      }
+
+      // Ensure parent exists
+      if (!fs.exists(tempTableDir.getParent()) && !fs.mkdirs(tempTableDir.getParent())) {
+        throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
+      }
+
+      // Move the table in /hbase/.tmp
+      if (!fs.rename(tableDir, tempTableDir)) {
+        if (fs.exists(tempTableDir)) {
+          // TODO
+          // what's in this dir? something old? probably something manual from the user...
+          // let's get rid of this stuff...
+          FileStatus[] files = fs.listStatus(tempdir);
+          if (files != null && files.length > 0) {
+            for (int i = 0; i < files.length; ++i) {
+              if (!files[i].isDir()) continue;
+              HFileArchiver.archiveRegion(fs, mfs.getRootDir(), tempTableDir, files[i].getPath());
+            }
+          }
+          fs.delete(tempdir, true);
+        }
+        throw new IOException("Unable to move '" + tableDir + "' to temp '" + tempTableDir + "'");
+      }
+    }
+
+    // Archive regions from FS (temp directory)
+    if (archive) {
+      for (HRegionInfo hri : regions) {
+        LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
+        HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
+            tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
+      }
+      LOG.debug("Table '" + tableName + "' archived!");
+    }
+
+    // Delete table directory from FS (temp directory)
+    if (!fs.delete(tempTableDir, true) && fs.exists(tempTableDir)) {
+      throw new IOException("Couldn't delete " + tempTableDir);
+    }
+  }
+
+  /**
+   * There may be items for this table still up in hbase:meta in the case where the
+   * info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta
+   * that have to do with this table. See HBASE-12980.
+   * @throws IOException
+   */
+  private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    ClusterConnection connection = env.getMasterServices().getConnection();
+    Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
+    try (Table metaTable =
+        connection.getTable(TableName.META_TABLE_NAME)) {
+      List<Delete> deletes = new ArrayList<Delete>();
+      try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
+        for (Result result : resScanner) {
+          deletes.add(new Delete(result.getRow()));
+        }
+      }
+      if (!deletes.isEmpty()) {
+        LOG.warn("Deleting some vestigal " + deletes.size() + " rows of " + tableName +
+          " from " + TableName.META_TABLE_NAME);
+        metaTable.delete(deletes);
+      }
+    }
+  }
+
+  protected static void deleteFromMeta(final MasterProcedureEnv env,
+      final TableName tableName, List<HRegionInfo> regions) throws IOException {
+    MetaTableAccessor.deleteRegions(env.getMasterServices().getConnection(), regions);
+
+    // Clean any remaining rows for this table.
+    cleanAnyRemainingRows(env, tableName);
+  }
+
+  protected static void deleteAssignmentState(final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+
+    // Clean up regions of the table in RegionStates.
+    LOG.debug("Removing '" + tableName + "' from region states.");
+    am.getRegionStates().tableDeleted(tableName);
+
+    // If entry for this table states, remove it.
+    LOG.debug("Marking '" + tableName + "' as deleted.");
+    am.getTableStateManager().setDeletedTable(tableName);
+  }
+
+  protected static void deleteTableDescriptorCache(final MasterProcedureEnv env,
+      final TableName tableName) throws IOException {
+    LOG.debug("Removing '" + tableName + "' descriptor.");
+    env.getMasterServices().getTableDescriptors().remove(tableName);
+  }
+
+  protected static void deleteTableStates(final MasterProcedureEnv env, final TableName tableName)
+      throws IOException {
+    getMasterQuotaManager(env).removeTableFromNamespaceQuota(tableName);
+  }
+
+  private static MasterQuotaManager getMasterQuotaManager(final MasterProcedureEnv env)
+      throws IOException {
+    return ProcedureSyncWait.waitFor(env, "quota manager to be available",
+        new ProcedureSyncWait.Predicate<MasterQuotaManager>() {
+      @Override
+      public MasterQuotaManager evaluate() throws IOException {
+        return env.getMasterServices().getMasterQuotaManager();
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
new file mode 100644
index 0000000..90ed4ee
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureConstants.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public final class MasterProcedureConstants {
+  private MasterProcedureConstants() {}
+
+  public static final String MASTER_PROCEDURE_LOGDIR = "MasterProcWALs";
+
+  public static final String MASTER_PROCEDURE_THREADS = "hbase.master.procedure.threads";
+  public static final int DEFAULT_MIN_MASTER_PROCEDURE_THREADS = 4;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
new file mode 100644
index 0000000..0a33cd4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MasterProcedureEnv {
+  private static final Log LOG = LogFactory.getLog(MasterProcedureEnv.class);
+
+  @InterfaceAudience.Private
+  public static class WALStoreLeaseRecovery implements WALProcedureStore.LeaseRecovery {
+    private final HMaster master;
+
+    public WALStoreLeaseRecovery(final HMaster master) {
+      this.master = master;
+    }
+
+    @Override
+    public void recoverFileLease(final FileSystem fs, final Path path) throws IOException {
+      final Configuration conf = master.getConfiguration();
+      final FSUtils fsUtils = FSUtils.getInstance(fs, conf);
+      fsUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
+        @Override
+        public boolean progress() {
+          LOG.debug("Recover Procedure Store log lease: " + path);
+          return master.isActiveMaster();
+        }
+      });
+    }
+  }
+
+  @InterfaceAudience.Private
+  public static class MasterProcedureStoreListener
+      implements ProcedureStore.ProcedureStoreListener {
+    private final HMaster master;
+
+    public MasterProcedureStoreListener(final HMaster master) {
+      this.master = master;
+    }
+
+    @Override
+    public void abortProcess() {
+      master.abort("The Procedure Store lost the lease");
+    }
+  }
+
+  private final MasterProcedureQueue procQueue;
+  private final MasterServices master;
+
+  public MasterProcedureEnv(final MasterServices master) {
+    this.master = master;
+    this.procQueue = new MasterProcedureQueue(master.getConfiguration(),
+      master.getTableLockManager());
+  }
+
+  public User getRequestUser() throws IOException {
+    User user = RpcServer.getRequestUser();
+    if (user == null) {
+      user = UserProvider.instantiate(getMasterConfiguration()).getCurrent();
+    }
+    return user;
+  }
+
+  public MasterServices getMasterServices() {
+    return master;
+  }
+
+  public Configuration getMasterConfiguration() {
+    return master.getConfiguration();
+  }
+
+  public MasterCoprocessorHost getMasterCoprocessorHost() {
+    return master.getMasterCoprocessorHost();
+  }
+
+  public MasterProcedureQueue getProcedureQueue() {
+    return procQueue;
+  }
+
+  public boolean isRunning() {
+    return master.getMasterProcedureExecutor().isRunning();
+  }
+
+  public boolean isInitialized() {
+    return master.isInitialized();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureQueue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureQueue.java
new file mode 100644
index 0000000..0dd0c3d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureQueue.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.util.ArrayDeque;
+import java.util.Deque;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureFairRunQueues;
+import org.apache.hadoop.hbase.procedure2.ProcedureRunnableSet;
+import org.apache.hadoop.hbase.master.TableLockManager;
+import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
+import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
+
+/**
+ * ProcedureRunnableSet for the Master Procedures.
+ * This RunnableSet tries to provide to the ProcedureExecutor procedures
+ * that can be executed without having to wait on a lock.
+ * Most of the master operations can be executed concurrently, if the they
+ * are operating on different tables (e.g. two create table can be performed
+ * at the same, time assuming table A and table B).
+ *
+ * Each procedure should implement an interface providing information for this queue.
+ * for example table related procedures should implement TableProcedureInterface.
+ * each procedure will be pushed in its own queue, and based on the operation type
+ * we may take smarter decision. e.g. we can abort all the operations preceding
+ * a delete table, or similar.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MasterProcedureQueue implements ProcedureRunnableSet {
+  private static final Log LOG = LogFactory.getLog(MasterProcedureQueue.class);
+
+  private final ProcedureFairRunQueues<TableName, RunQueue> fairq;
+  private final ReentrantLock lock = new ReentrantLock();
+  private final Condition waitCond = lock.newCondition();
+  private final TableLockManager lockManager;
+
+  private final int metaTablePriority;
+  private final int userTablePriority;
+  private final int sysTablePriority;
+
+  private int queueSize;
+
+  public MasterProcedureQueue(final Configuration conf, final TableLockManager lockManager) {
+    this.fairq = new ProcedureFairRunQueues<TableName, RunQueue>(1);
+    this.lockManager = lockManager;
+
+    // TODO: should this be part of the HTD?
+    metaTablePriority = conf.getInt("hbase.master.procedure.queue.meta.table.priority", 3);
+    sysTablePriority = conf.getInt("hbase.master.procedure.queue.system.table.priority", 2);
+    userTablePriority = conf.getInt("hbase.master.procedure.queue.user.table.priority", 1);
+  }
+
+  @Override
+  public void addFront(final Procedure proc) {
+    lock.lock();
+    try {
+      getRunQueueOrCreate(proc).addFront(proc);
+      queueSize++;
+      waitCond.signal();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void addBack(final Procedure proc) {
+    lock.lock();
+    try {
+      getRunQueueOrCreate(proc).addBack(proc);
+      queueSize++;
+      waitCond.signal();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void yield(final Procedure proc) {
+    addFront(proc);
+  }
+
+  @Override
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
+  public Long poll() {
+    lock.lock();
+    try {
+      if (queueSize == 0) {
+        waitCond.await();
+        if (queueSize == 0) {
+          return null;
+        }
+      }
+
+      RunQueue queue = fairq.poll();
+      if (queue != null && queue.isAvailable()) {
+        queueSize--;
+        return queue.poll();
+      }
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return null;
+    } finally {
+      lock.unlock();
+    }
+    return null;
+  }
+
+  @Override
+  public void signalAll() {
+    lock.lock();
+    try {
+      waitCond.signalAll();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void clear() {
+    lock.lock();
+    try {
+      fairq.clear();
+      queueSize = 0;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public int size() {
+    lock.lock();
+    try {
+      return queueSize;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    lock.lock();
+    try {
+      return "MasterProcedureQueue size=" + queueSize + ": " + fairq;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void completionCleanup(Procedure proc) {
+    if (proc instanceof TableProcedureInterface) {
+      TableProcedureInterface iProcTable = (TableProcedureInterface)proc;
+      boolean tableDeleted;
+      if (proc.hasException()) {
+        IOException procEx =  proc.getException().unwrapRemoteException();
+        if (iProcTable.getTableOperationType() == TableOperationType.CREATE) {
+          // create failed because the table already exist
+          tableDeleted = !(procEx instanceof TableExistsException);
+        } else {
+          // the operation failed because the table does not exist
+          tableDeleted = (procEx instanceof TableNotFoundException);
+        }
+      } else {
+        // the table was deleted
+        tableDeleted = (iProcTable.getTableOperationType() == TableOperationType.DELETE);
+      }
+      if (tableDeleted) {
+        markTableAsDeleted(iProcTable.getTableName());
+      }
+    }
+  }
+
+  private RunQueue getRunQueueOrCreate(final Procedure proc) {
+    if (proc instanceof TableProcedureInterface) {
+      final TableName table = ((TableProcedureInterface)proc).getTableName();
+      return getRunQueueOrCreate(table);
+    }
+    // TODO: at the moment we only have Table procedures
+    // if you are implementing a non-table procedure, you have two option create
+    // a group for all the non-table procedures or try to find a key for your
+    // non-table procedure and implement something similar to the TableRunQueue.
+    throw new UnsupportedOperationException("RQs for non-table procedures are not implemented yet");
+  }
+
+  private TableRunQueue getRunQueueOrCreate(final TableName table) {
+    final TableRunQueue queue = getRunQueue(table);
+    if (queue != null) return queue;
+    return (TableRunQueue)fairq.add(table, createTableRunQueue(table));
+  }
+
+  private TableRunQueue createTableRunQueue(final TableName table) {
+    int priority = userTablePriority;
+    if (table.equals(TableName.META_TABLE_NAME)) {
+      priority = metaTablePriority;
+    } else if (table.isSystemTable()) {
+      priority = sysTablePriority;
+    }
+    return new TableRunQueue(priority);
+  }
+
+  private TableRunQueue getRunQueue(final TableName table) {
+    return (TableRunQueue)fairq.get(table);
+  }
+
+  /**
+   * Try to acquire the read lock on the specified table.
+   * other read operations in the table-queue may be executed concurrently,
+   * otherwise they have to wait until all the read-locks are released.
+   * @param table Table to lock
+   * @param purpose Human readable reason for locking the table
+   * @return true if we were able to acquire the lock on the table, otherwise false.
+   */
+  public boolean tryAcquireTableRead(final TableName table, final String purpose) {
+    return getRunQueueOrCreate(table).tryRead(lockManager, table, purpose);
+  }
+
+  /**
+   * Release the read lock taken with tryAcquireTableRead()
+   * @param table the name of the table that has the read lock
+   */
+  public void releaseTableRead(final TableName table) {
+    getRunQueue(table).releaseRead(lockManager, table);
+  }
+
+  /**
+   * Try to acquire the write lock on the specified table.
+   * other operations in the table-queue will be executed after the lock is released.
+   * @param table Table to lock
+   * @param purpose Human readable reason for locking the table
+   * @return true if we were able to acquire the lock on the table, otherwise false.
+   */
+  public boolean tryAcquireTableWrite(final TableName table, final String purpose) {
+    return getRunQueueOrCreate(table).tryWrite(lockManager, table, purpose);
+  }
+
+  /**
+   * Release the write lock taken with tryAcquireTableWrite()
+   * @param table the name of the table that has the write lock
+   */
+  public void releaseTableWrite(final TableName table) {
+    getRunQueue(table).releaseWrite(lockManager, table);
+  }
+
+  /**
+   * Tries to remove the queue and the table-lock of the specified table.
+   * If there are new operations pending (e.g. a new create),
+   * the remove will not be performed.
+   * @param table the name of the table that should be marked as deleted
+   * @return true if deletion succeeded, false otherwise meaning that there are
+   *    other new operations pending for that table (e.g. a new create).
+   */
+  protected boolean markTableAsDeleted(final TableName table) {
+    TableRunQueue queue = getRunQueue(table);
+    if (queue != null) {
+      lock.lock();
+      try {
+        if (queue.isEmpty() && !queue.isLocked()) {
+          fairq.remove(table);
+
+          // Remove the table lock
+          try {
+            lockManager.tableDeleted(table);
+          } catch (IOException e) {
+            LOG.warn("Received exception from TableLockManager.tableDeleted:", e); //not critical
+          }
+        } else {
+          // TODO: If there are no create, we can drop all the other ops
+          return false;
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+    return true;
+  }
+
+  private interface RunQueue extends ProcedureFairRunQueues.FairObject {
+    void addFront(Procedure proc);
+    void addBack(Procedure proc);
+    Long poll();
+    boolean isLocked();
+  }
+
+  /**
+   * Run Queue for a Table. It contains a read-write lock that is used by the
+   * MasterProcedureQueue to decide if we should fetch an item from this queue
+   * or skip to another one which will be able to run without waiting for locks.
+   */
+  private static class TableRunQueue implements RunQueue {
+    private final Deque<Long> runnables = new ArrayDeque<Long>();
+    private final int priority;
+
+    private TableLock tableLock = null;
+    private boolean wlock = false;
+    private int rlock = 0;
+
+    public TableRunQueue(int priority) {
+      this.priority = priority;
+    }
+
+    @Override
+    public void addFront(final Procedure proc) {
+      runnables.addFirst(proc.getProcId());
+    }
+
+    // TODO: Improve run-queue push with TableProcedureInterface.getType()
+    //       we can take smart decisions based on the type of the operation (e.g. create/delete)
+    @Override
+    public void addBack(final Procedure proc) {
+      runnables.addLast(proc.getProcId());
+    }
+
+    @Override
+    public Long poll() {
+      return runnables.poll();
+    }
+
+    @Override
+    public boolean isAvailable() {
+      synchronized (this) {
+        return !wlock && !runnables.isEmpty();
+      }
+    }
+
+    public boolean isEmpty() {
+      return runnables.isEmpty();
+    }
+
+    @Override
+    public boolean isLocked() {
+      synchronized (this) {
+        return wlock || rlock > 0;
+      }
+    }
+
+    public boolean tryRead(final TableLockManager lockManager,
+        final TableName tableName, final String purpose) {
+      synchronized (this) {
+        if (wlock) {
+          return false;
+        }
+
+        // Take zk-read-lock
+        tableLock = lockManager.readLock(tableName, purpose);
+        try {
+          tableLock.acquire();
+        } catch (IOException e) {
+          LOG.error("failed acquire read lock on " + tableName, e);
+          tableLock = null;
+          return false;
+        }
+
+        rlock++;
+      }
+      return true;
+    }
+
+    public void releaseRead(final TableLockManager lockManager,
+        final TableName tableName) {
+      synchronized (this) {
+        releaseTableLock(lockManager, rlock == 1);
+        rlock--;
+      }
+    }
+
+    public boolean tryWrite(final TableLockManager lockManager,
+        final TableName tableName, final String purpose) {
+      synchronized (this) {
+        if (wlock || rlock > 0) {
+          return false;
+        }
+
+        // Take zk-write-lock
+        tableLock = lockManager.writeLock(tableName, purpose);
+        try {
+          tableLock.acquire();
+        } catch (IOException e) {
+          LOG.error("failed acquire write lock on " + tableName, e);
+          tableLock = null;
+          return false;
+        }
+        wlock = true;
+      }
+      return true;
+    }
+
+    public void releaseWrite(final TableLockManager lockManager,
+        final TableName tableName) {
+      synchronized (this) {
+        releaseTableLock(lockManager, true);
+        wlock = false;
+      }
+    }
+
+    private void releaseTableLock(final TableLockManager lockManager, boolean reset) {
+      for (int i = 0; i < 3; ++i) {
+        try {
+          tableLock.release();
+          if (reset) {
+            tableLock = null;
+          }
+          break;
+        } catch (IOException e) {
+          LOG.warn("Could not release the table write-lock", e);
+        }
+      }
+    }
+
+    @Override
+    public int getPriority() {
+      return priority;
+    }
+
+    @Override
+    public String toString() {
+      return runnables.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
new file mode 100644
index 0000000..d7c0b92
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class MasterProcedureUtil {
+  private static final Log LOG = LogFactory.getLog(MasterProcedureUtil.class);
+
+  private MasterProcedureUtil() {}
+
+  public static UserInformation toProtoUserInfo(UserGroupInformation ugi) {
+    UserInformation.Builder userInfoPB = UserInformation.newBuilder();
+    userInfoPB.setEffectiveUser(ugi.getUserName());
+    if (ugi.getRealUser() != null) {
+      userInfoPB.setRealUser(ugi.getRealUser().getUserName());
+    }
+    return userInfoPB.build();
+  }
+
+  public static UserGroupInformation toUserInfo(UserInformation userInfoProto) {
+    if (userInfoProto.hasEffectiveUser()) {
+      String effectiveUser = userInfoProto.getEffectiveUser();
+      if (userInfoProto.hasRealUser()) {
+        String realUser = userInfoProto.getRealUser();
+        UserGroupInformation realUserUgi = UserGroupInformation.createRemoteUser(realUser);
+        return UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
+      }
+      return UserGroupInformation.createRemoteUser(effectiveUser);
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
new file mode 100644
index 0000000..2a1abca
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -0,0 +1,105 @@
+/**
+ * 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.InterruptedIOException;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.ipc.RpcCallContext;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
+
+/**
+ * Latch used by the Master to have the prepare() sync behaviour for old
+ * clients, that can only get exceptions in a synchronous way.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class ProcedurePrepareLatch {
+  private static final NoopLatch noopLatch = new NoopLatch();
+
+  public static ProcedurePrepareLatch createLatch() {
+    // don't use the latch if we have procedure support
+    return hasProcedureSupport() ? noopLatch : new CompatibilityLatch();
+  }
+
+  public static boolean hasProcedureSupport() {
+    return currentClientHasMinimumVersion(1, 1);
+  }
+
+  private static boolean currentClientHasMinimumVersion(int major, int minor) {
+    RpcCallContext call = RpcServer.getCurrentCall();
+    VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
+    if (versionInfo != null) {
+      String[] components = versionInfo.getVersion().split("\\.");
+
+      int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
+      if (clientMajor != major) {
+        return clientMajor > major;
+      }
+
+      int clientMinor = components.length > 1 ? Integer.parseInt(components[1]) : 0;
+      return clientMinor >= minor;
+    }
+    return false;
+  }
+
+  protected abstract void countDown(final Procedure proc);
+  public abstract void await() throws IOException;
+
+  protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
+    if (latch != null) {
+      latch.countDown(proc);
+    }
+  }
+
+  private static class NoopLatch extends ProcedurePrepareLatch {
+    protected void countDown(final Procedure proc) {}
+    public void await() throws IOException {}
+  }
+
+  protected static class CompatibilityLatch extends ProcedurePrepareLatch {
+    private final CountDownLatch latch = new CountDownLatch(1);
+
+    private IOException exception = null;
+
+    protected void countDown(final Procedure proc) {
+      if (proc.hasException()) {
+        exception = proc.getException().unwrapRemoteException();
+      }
+      latch.countDown();
+    }
+
+    public void await() throws IOException {
+      try {
+        latch.await();
+      } catch (InterruptedException e) {
+        throw (InterruptedIOException)new InterruptedIOException().initCause(e);
+      }
+
+      if (exception != null) {
+        throw exception;
+      }
+    }
+  }
+}


[08/18] hbase git commit: HBASE-13204 Procedure v2 - client create/delete table sync

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 37f726c..c30d92a 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 import "HBase.proto";
 import "Client.proto";
 import "ClusterStatus.proto";
+import "ErrorHandling.proto";
 import "Quota.proto";
 
 /* Column-level protobufs */
@@ -108,6 +109,7 @@ message CreateTableRequest {
 }
 
 message CreateTableResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DeleteTableRequest {
@@ -115,6 +117,7 @@ message DeleteTableRequest {
 }
 
 message DeleteTableResponse {
+  optional uint64 proc_id = 1;
 }
 
 message TruncateTableRequest {
@@ -380,6 +383,24 @@ message IsProcedureDoneResponse {
         optional ProcedureDescription snapshot = 2;
 }
 
+message GetProcedureResultRequest {
+  required uint64 proc_id = 1;
+}
+
+message GetProcedureResultResponse {
+  enum State {
+    NOT_FOUND = 0;
+    RUNNING = 1;
+    FINISHED = 2;
+  }
+
+  required State state = 1;
+  optional uint64 start_time = 2;
+  optional uint64 last_update = 3;
+  optional bytes result = 4;
+  optional ForeignExceptionMessage exception = 5;
+}
+
 message SetQuotaRequest {
   optional string user_name = 1;
   optional string user_group = 2;
@@ -634,4 +655,7 @@ service MasterService {
   /** Returns the timestamp of the last major compaction */
   rpc getLastMajorCompactionTimestampForRegion(MajorCompactionTimestampForRegionRequest)
     returns(MajorCompactionTimestampResponse);
+
+  rpc getProcedureResult(GetProcedureResultRequest)
+    returns(GetProcedureResultResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 8ec883a..e2e600c 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
@@ -1326,7 +1326,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void createTable(HTableDescriptor hTableDescriptor,
+  public long createTable(HTableDescriptor hTableDescriptor,
       byte [][] splitKeys) throws IOException {
     if (isStopped()) {
       throw new MasterNotRunningException();
@@ -1357,9 +1357,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.postCreateTable(hTableDescriptor, newRegions);
     }
 
-    // TODO: change the interface to return the procId,
-    //       and add it to the response protobuf.
-    //return procId;
+    return procId;
   }
 
   /**
@@ -1571,7 +1569,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void deleteTable(final TableName tableName) throws IOException {
+  public long deleteTable(final TableName tableName) throws IOException {
     checkInitialized();
     if (cpHost != null) {
       cpHost.preDeleteTable(tableName);
@@ -1588,9 +1586,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       cpHost.postDeleteTable(tableName);
     }
 
-    // TODO: change the interface to return the procId,
-    //       and add it to the response protobuf.
-    //return procId;
+    return procId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 07b2da2..abdbf5a 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
@@ -43,6 +43,8 @@ import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManager;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureResult;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
@@ -86,6 +88,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnaps
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@@ -158,6 +162,7 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 
@@ -405,11 +410,11 @@ public class MasterRpcServices extends RSRpcServices
     HTableDescriptor hTableDescriptor = HTableDescriptor.convert(req.getTableSchema());
     byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
     try {
-      master.createTable(hTableDescriptor, splitKeys);
+      long procId = master.createTable(hTableDescriptor, splitKeys);
+      return CreateTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
-    return CreateTableResponse.newBuilder().build();
   }
 
   @Override
@@ -461,11 +466,11 @@ public class MasterRpcServices extends RSRpcServices
   public DeleteTableResponse deleteTable(RpcController controller,
       DeleteTableRequest request) throws ServiceException {
     try {
-      master.deleteTable(ProtobufUtil.toTableName(request.getTableName()));
+      long procId = master.deleteTable(ProtobufUtil.toTableName(request.getTableName()));
+      return DeleteTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
-    return DeleteTableResponse.newBuilder().build();
   }
 
   @Override
@@ -962,6 +967,44 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public GetProcedureResultResponse getProcedureResult(RpcController controller,
+      GetProcedureResultRequest request) throws ServiceException {
+    LOG.debug("Checking to see if procedure is done procId=" + request.getProcId());
+    try {
+      master.checkInitialized();
+      GetProcedureResultResponse.Builder builder = GetProcedureResultResponse.newBuilder();
+
+      Pair<ProcedureResult, Procedure> v = master.getMasterProcedureExecutor()
+          .getResultOrProcedure(request.getProcId());
+      if (v.getFirst() != null) {
+        ProcedureResult result = v.getFirst();
+        builder.setState(GetProcedureResultResponse.State.FINISHED);
+        builder.setStartTime(result.getStartTime());
+        builder.setLastUpdate(result.getLastUpdate());
+        if (result.isFailed()) {
+          builder.setException(result.getException().convert());
+        }
+        if (result.hasResultData()) {
+          builder.setResult(ByteStringer.wrap(result.getResult()));
+        }
+        master.getMasterProcedureExecutor().removeResult(request.getProcId());
+      } else {
+        Procedure proc = v.getSecond();
+        if (proc == null) {
+          builder.setState(GetProcedureResultResponse.State.NOT_FOUND);
+        } else {
+          builder.setState(GetProcedureResultResponse.State.RUNNING);
+          builder.setStartTime(proc.getStartTime());
+          builder.setLastUpdate(proc.getLastUpdate());
+        }
+      }
+      return builder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public ListNamespaceDescriptorsResponse listNamespaceDescriptors(RpcController c,
       ListNamespaceDescriptorsRequest request) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 7352fe8..d823b35 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
@@ -105,7 +105,7 @@ public interface MasterServices extends Server {
    * @param splitKeys Starting row keys for the initial table regions.  If null
    *     a single region is created.
    */
-  void createTable(HTableDescriptor desc, byte[][] splitKeys)
+  long createTable(HTableDescriptor desc, byte[][] splitKeys)
       throws IOException;
 
   /**
@@ -113,7 +113,7 @@ public interface MasterServices extends Server {
    * @param tableName The table name
    * @throws IOException
    */
-  void deleteTable(final TableName tableName) throws IOException;
+  long deleteTable(final TableName tableName) throws IOException;
 
   /**
    * Truncate a table

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 2c13f39..5b2e50d 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
@@ -227,9 +227,10 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void createTable(HTableDescriptor desc, byte[][] splitKeys)
+    public long createTable(HTableDescriptor desc, byte[][] splitKeys)
         throws IOException {
       // no-op
+      return -1;
     }
 
     @Override
@@ -427,7 +428,9 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void deleteTable(TableName tableName) throws IOException { }
+    public long deleteTable(TableName tableName) throws IOException {
+      return -1;
+    }
 
     @Override
     public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { }


[11/18] 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/5946a525/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/5946a525/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();
+  }
+}


[15/18] 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/6fbf41dd/hbase-protocol/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/MasterProcedure.proto b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
index f09dc98..012b279 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -139,3 +139,33 @@ message DeleteColumnFamilyMessage {
   optional TableSchema unmodified_table_schema = 4;
   repeated RegionInfo region_info = 5;
 }
+
+enum EnableTableState {
+  ENABLE_TABLE_PREPARE = 1;
+  ENABLE_TABLE_PRE_OPERATION = 2;
+  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
+  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
+  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
+  ENABLE_TABLE_POST_OPERATION = 6;
+}
+
+message EnableTableMessage {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+enum DisableTableState {
+  DISABLE_TABLE_PREPARE = 1;
+  DISABLE_TABLE_PRE_OPERATION = 2;
+  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
+  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
+  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
+  DISABLE_TABLE_POST_OPERATION = 6;
+}
+
+message DisableTableMessage {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}

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

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

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

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

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

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

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

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


[03/18] hbase git commit: HBASE-13203 Procedure v2 - master create/delete table

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/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
new file mode 100644
index 0000000..6d1694a
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProcedureProtos.java
@@ -0,0 +1,2633 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: MasterProcedure.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class MasterProcedureProtos {
+  private MasterProcedureProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  /**
+   * Protobuf enum {@code CreateTableState}
+   */
+  public enum CreateTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>CREATE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    CREATE_TABLE_PRE_OPERATION(0, 1),
+    /**
+     * <code>CREATE_TABLE_WRITE_FS_LAYOUT = 2;</code>
+     */
+    CREATE_TABLE_WRITE_FS_LAYOUT(1, 2),
+    /**
+     * <code>CREATE_TABLE_ADD_TO_META = 3;</code>
+     */
+    CREATE_TABLE_ADD_TO_META(2, 3),
+    /**
+     * <code>CREATE_TABLE_ASSIGN_REGIONS = 4;</code>
+     */
+    CREATE_TABLE_ASSIGN_REGIONS(3, 4),
+    /**
+     * <code>CREATE_TABLE_UPDATE_DESC_CACHE = 5;</code>
+     */
+    CREATE_TABLE_UPDATE_DESC_CACHE(4, 5),
+    /**
+     * <code>CREATE_TABLE_POST_OPERATION = 6;</code>
+     */
+    CREATE_TABLE_POST_OPERATION(5, 6),
+    ;
+
+    /**
+     * <code>CREATE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    public static final int CREATE_TABLE_PRE_OPERATION_VALUE = 1;
+    /**
+     * <code>CREATE_TABLE_WRITE_FS_LAYOUT = 2;</code>
+     */
+    public static final int CREATE_TABLE_WRITE_FS_LAYOUT_VALUE = 2;
+    /**
+     * <code>CREATE_TABLE_ADD_TO_META = 3;</code>
+     */
+    public static final int CREATE_TABLE_ADD_TO_META_VALUE = 3;
+    /**
+     * <code>CREATE_TABLE_ASSIGN_REGIONS = 4;</code>
+     */
+    public static final int CREATE_TABLE_ASSIGN_REGIONS_VALUE = 4;
+    /**
+     * <code>CREATE_TABLE_UPDATE_DESC_CACHE = 5;</code>
+     */
+    public static final int CREATE_TABLE_UPDATE_DESC_CACHE_VALUE = 5;
+    /**
+     * <code>CREATE_TABLE_POST_OPERATION = 6;</code>
+     */
+    public static final int CREATE_TABLE_POST_OPERATION_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static CreateTableState valueOf(int value) {
+      switch (value) {
+        case 1: return CREATE_TABLE_PRE_OPERATION;
+        case 2: return CREATE_TABLE_WRITE_FS_LAYOUT;
+        case 3: return CREATE_TABLE_ADD_TO_META;
+        case 4: return CREATE_TABLE_ASSIGN_REGIONS;
+        case 5: return CREATE_TABLE_UPDATE_DESC_CACHE;
+        case 6: return CREATE_TABLE_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<CreateTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<CreateTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<CreateTableState>() {
+            public CreateTableState findValueByNumber(int number) {
+              return CreateTableState.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(0);
+    }
+
+    private static final CreateTableState[] VALUES = values();
+
+    public static CreateTableState 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 CreateTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:CreateTableState)
+  }
+
+  /**
+   * Protobuf enum {@code DeleteTableState}
+   */
+  public enum DeleteTableState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>DELETE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    DELETE_TABLE_PRE_OPERATION(0, 1),
+    /**
+     * <code>DELETE_TABLE_REMOVE_FROM_META = 2;</code>
+     */
+    DELETE_TABLE_REMOVE_FROM_META(1, 2),
+    /**
+     * <code>DELETE_TABLE_CLEAR_FS_LAYOUT = 3;</code>
+     */
+    DELETE_TABLE_CLEAR_FS_LAYOUT(2, 3),
+    /**
+     * <code>DELETE_TABLE_UPDATE_DESC_CACHE = 4;</code>
+     */
+    DELETE_TABLE_UPDATE_DESC_CACHE(3, 4),
+    /**
+     * <code>DELETE_TABLE_UNASSIGN_REGIONS = 5;</code>
+     */
+    DELETE_TABLE_UNASSIGN_REGIONS(4, 5),
+    /**
+     * <code>DELETE_TABLE_POST_OPERATION = 6;</code>
+     */
+    DELETE_TABLE_POST_OPERATION(5, 6),
+    ;
+
+    /**
+     * <code>DELETE_TABLE_PRE_OPERATION = 1;</code>
+     */
+    public static final int DELETE_TABLE_PRE_OPERATION_VALUE = 1;
+    /**
+     * <code>DELETE_TABLE_REMOVE_FROM_META = 2;</code>
+     */
+    public static final int DELETE_TABLE_REMOVE_FROM_META_VALUE = 2;
+    /**
+     * <code>DELETE_TABLE_CLEAR_FS_LAYOUT = 3;</code>
+     */
+    public static final int DELETE_TABLE_CLEAR_FS_LAYOUT_VALUE = 3;
+    /**
+     * <code>DELETE_TABLE_UPDATE_DESC_CACHE = 4;</code>
+     */
+    public static final int DELETE_TABLE_UPDATE_DESC_CACHE_VALUE = 4;
+    /**
+     * <code>DELETE_TABLE_UNASSIGN_REGIONS = 5;</code>
+     */
+    public static final int DELETE_TABLE_UNASSIGN_REGIONS_VALUE = 5;
+    /**
+     * <code>DELETE_TABLE_POST_OPERATION = 6;</code>
+     */
+    public static final int DELETE_TABLE_POST_OPERATION_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static DeleteTableState valueOf(int value) {
+      switch (value) {
+        case 1: return DELETE_TABLE_PRE_OPERATION;
+        case 2: return DELETE_TABLE_REMOVE_FROM_META;
+        case 3: return DELETE_TABLE_CLEAR_FS_LAYOUT;
+        case 4: return DELETE_TABLE_UPDATE_DESC_CACHE;
+        case 5: return DELETE_TABLE_UNASSIGN_REGIONS;
+        case 6: return DELETE_TABLE_POST_OPERATION;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<DeleteTableState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<DeleteTableState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<DeleteTableState>() {
+            public DeleteTableState findValueByNumber(int number) {
+              return DeleteTableState.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 DeleteTableState[] VALUES = values();
+
+    public static DeleteTableState 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 DeleteTableState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:DeleteTableState)
+  }
+
+  public interface CreateTableStateDataOrBuilder
+      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 .TableSchema table_schema = 2;
+    /**
+     * <code>required .TableSchema table_schema = 2;</code>
+     */
+    boolean hasTableSchema();
+    /**
+     * <code>required .TableSchema table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getTableSchema();
+    /**
+     * <code>required .TableSchema table_schema = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder();
+
+    // repeated .RegionInfo region_info = 3;
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code CreateTableStateData}
+   */
+  public static final class CreateTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements CreateTableStateDataOrBuilder {
+    // Use CreateTableStateData.newBuilder() to construct.
+    private CreateTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private CreateTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final CreateTableStateData defaultInstance;
+    public static CreateTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public CreateTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private CreateTableStateData(
+        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 = tableSchema_.toBuilder();
+              }
+              tableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableSchema_);
+                tableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              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_ & 0x00000004) == 0x00000004)) {
+          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_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);
+    }
+
+    public static com.google.protobuf.Parser<CreateTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<CreateTableStateData>() {
+      public CreateTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new CreateTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<CreateTableStateData> 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 .TableSchema table_schema = 2;
+    public static final int TABLE_SCHEMA_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema tableSchema_;
+    /**
+     * <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() {
+      return tableSchema_;
+    }
+    /**
+     * <code>required .TableSchema table_schema = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getTableSchemaOrBuilder() {
+      return tableSchema_;
+    }
+
+    // repeated .RegionInfo region_info = 3;
+    public static final int REGION_INFO_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <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 DeleteTableStateDataOrBuilder
+      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();
+
+    // repeated .RegionInfo region_info = 3;
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
+        getRegionInfoList();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    int getRegionInfoCount();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList();
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code DeleteTableStateData}
+   */
+  public static final class DeleteTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements DeleteTableStateDataOrBuilder {
+    // Use DeleteTableStateData.newBuilder() to construct.
+    private DeleteTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private DeleteTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final DeleteTableStateData defaultInstance;
+    public static DeleteTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public DeleteTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private DeleteTableStateData(
+        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: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                regionInfo_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              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_ & 0x00000004) == 0x00000004)) {
+          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_DeleteTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<DeleteTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<DeleteTableStateData>() {
+      public DeleteTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new DeleteTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<DeleteTableStateData> 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_;
+    }
+
+    // repeated .RegionInfo region_info = 3;
+    public static final int REGION_INFO_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> regionInfo_;
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
+      return regionInfo_;
+    }
+    /**
+     * <code>repeated .RegionInfo region_info = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
+        getRegionInfoOrBuilderList() {
+      return regionInfo_;
+    }
+    /**
+     * <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();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.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 (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().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_);
+      }
+      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, tableName_);
+      }
+      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.DeleteTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData) 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 && 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 (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.DeleteTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData 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.DeleteTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData 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.DeleteTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData 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.DeleteTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData 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.DeleteTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData 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.DeleteTableStateData 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 DeleteTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.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();
+          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 (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_DeleteTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData(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 (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.DeleteTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        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 (!hasTableName()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().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.DeleteTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData) 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);
+      }
+      /

<TRUNCATED>

[07/18] 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/711ae17a
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/711ae17a
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/711ae17a

Branch: refs/heads/hbase-12439
Commit: 711ae17a3965ec6a84e16ad2f664c0b69f29d760
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 22:44:59 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 1935 ++++++++++++++++--
 .../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, 2914 insertions(+), 173 deletions(-)
----------------------------------------------------------------------



[12/18] 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/5946a525/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 93af886..f09dc98 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 AddColumnFamilyMessage {
+  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 ModifyColumnFamilyMessage {
+  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 DeleteColumnFamilyMessage {
+  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/5946a525/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/5946a525/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/5946a525/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..2756d09
--- /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.AddColumnFamilyMessage.Builder addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyMessage.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.AddColumnFamilyMessage addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyMessage.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/5946a525/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..9a424e0
--- /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.DeleteColumnFamilyMessage.Builder deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyMessage.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.DeleteColumnFamilyMessage deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyMessage.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/5946a525/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..c9f750a
--- /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.ModifyColumnFamilyMessage.Builder modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyMessage.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.ModifyColumnFamilyMessage modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyMessage.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/5946a525/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/5946a525/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/5946a525/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/5946a525/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/5946a525/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();
+  }
+}


[18/18] 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/879f805d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/879f805d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/879f805d

Branch: refs/heads/hbase-12439
Commit: 879f805d79e101a17a6316182340f093f9abd2ba
Parents: 6fbf41d
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 22:45:00 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/879f805d/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/879f805d/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/879f805d/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/879f805d/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/879f805d/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/879f805d/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/879f805d/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)


[05/18] 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/711ae17a/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..93af886 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 ModifyTableMessage {
+  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/711ae17a/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/711ae17a/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/711ae17a/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/711ae17a/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..f09b686
--- /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.ModifyTableMessage.Builder modifyTableMsg =
+        MasterProcedureProtos.ModifyTableMessage.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.ModifyTableMessage modifyTableMsg =
+        MasterProcedureProtos.ModifyTableMessage.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/711ae17a/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/18] hbase git commit: HBASE-13203 Procedure v2 - master create/delete table

Posted by mb...@apache.org.
HBASE-13203 Procedure v2 - master create/delete table


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

Branch: refs/heads/hbase-12439
Commit: aa934f835f30ec208dc085688169fcf7d7f420ac
Parents: 9d763cd
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 20:47:46 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 22:44:55 2015 +0100

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |    4 +-
 .../hbase/exceptions/TimeoutIOException.java    |   46 +
 hbase-protocol/pom.xml                          |    1 +
 .../generated/MasterProcedureProtos.java        | 2633 ++++++++++++++++++
 .../src/main/protobuf/MasterProcedure.proto     |   74 +
 hbase-server/pom.xml                            |   10 +
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |    6 +
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |   15 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  106 +-
 .../hadoop/hbase/master/MasterServices.java     |    7 +
 .../hbase/master/TableNamespaceManager.java     |   19 +-
 .../master/procedure/CreateTableProcedure.java  |  442 +++
 .../master/procedure/DeleteTableProcedure.java  |  420 +++
 .../procedure/MasterProcedureConstants.java     |   31 +
 .../master/procedure/MasterProcedureEnv.java    |  123 +
 .../master/procedure/MasterProcedureQueue.java  |  448 +++
 .../master/procedure/MasterProcedureUtil.java   |   56 +
 .../master/procedure/ProcedurePrepareLatch.java |  105 +
 .../master/procedure/ProcedureSyncWait.java     |  179 ++
 .../procedure/TableProcedureInterface.java      |   46 +
 .../hadoop/hbase/quotas/MasterQuotaManager.java |   15 +-
 .../hbase/regionserver/HRegionServer.java       |    9 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |   24 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |    9 +-
 .../MasterProcedureTestingUtility.java          |  317 +++
 .../procedure/TestCreateTableProcedure.java     |  257 ++
 .../procedure/TestDeleteTableProcedure.java     |  208 ++
 .../TestMasterFailoverWithProcedures.java       |  291 ++
 .../procedure/TestMasterProcedureQueue.java     |  433 +++
 29 files changed, 6280 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index d18239b..ea29e4f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -249,8 +249,10 @@ public class MetaTableAccessor {
   static Table getMetaHTable(final Connection connection)
   throws IOException {
     // We used to pass whole CatalogTracker in here, now we just pass in Connection
-    if (connection == null || connection.isClosed()) {
+    if (connection == null) {
       throw new NullPointerException("No connection");
+    } else if (connection.isClosed()) {
+      throw new IOException("connection is closed");
     }
     return connection.getTable(TableName.META_TABLE_NAME);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TimeoutIOException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TimeoutIOException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TimeoutIOException.java
new file mode 100644
index 0000000..4e1ee39
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/exceptions/TimeoutIOException.java
@@ -0,0 +1,46 @@
+/**
+ * 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.exceptions;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Exception thrown when a blocking operation times out.
+ */
+@SuppressWarnings("serial")
+@InterfaceAudience.Private
+public class TimeoutIOException extends IOException {
+  public TimeoutIOException() {
+    super();
+  }
+
+  public TimeoutIOException(final String message) {
+    super(message);
+  }
+
+  public TimeoutIOException(final String message, final Throwable t) {
+    super(message, t);
+  }
+
+  public TimeoutIOException(final Throwable t) {
+    super(t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aa934f83/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 0d33332..fb5e0ab 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -175,6 +175,7 @@
                           <include>LoadBalancer.proto</include>
                           <include>MapReduce.proto</include>
                           <include>Master.proto</include>
+                          <include>MasterProcedure.proto</include>
                           <include>MultiRowMutation.proto</include>
                           <include>Procedure.proto</include>
                           <include>Quota.proto</include>


[10/18] hbase git commit: HBASE-13204 Procedure v2 - client create/delete table sync

Posted by mb...@apache.org.
HBASE-13204 Procedure v2 - client create/delete table sync


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

Branch: refs/heads/hbase-12439
Commit: 02c15a2fed5a9196e3d3d86b68bf4d3dc88ec0e9
Parents: aa934f8
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:01:20 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Thu Apr 9 22:44:59 2015 +0100

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |    6 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  608 ++++-
 .../hbase/client/TestProcedureFuture.java       |  186 ++
 .../hbase/protobuf/generated/MasterProtos.java  | 2576 +++++++++++++++---
 hbase-protocol/src/main/protobuf/Master.proto   |   24 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   12 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   51 +-
 .../hadoop/hbase/master/MasterServices.java     |    4 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |    7 +-
 9 files changed, 2920 insertions(+), 554 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 8442a77..bc2d51a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -1598,6 +1598,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       }
 
       @Override
+      public MasterProtos.GetProcedureResultResponse getProcedureResult(RpcController controller,
+          MasterProtos.GetProcedureResultRequest request) throws ServiceException {
+        return stub.getProcedureResult(controller, request);
+      }
+
+      @Override
       public MasterProtos.IsMasterRunningResponse isMasterRunning(
           RpcController controller, MasterProtos.IsMasterRunningRequest request)
           throws ServiceException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 21a9139..1697c03 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
@@ -31,6 +31,10 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
@@ -62,6 +66,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
 import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
@@ -89,10 +94,12 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
 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.DispatchMergingRegionsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
@@ -101,6 +108,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResp
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
@@ -142,6 +151,7 @@ import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -186,6 +196,7 @@ public class HBaseAdmin implements Admin {
   // numRetries is for 'normal' stuff... Multiply by this factor when
   // want to wait a long time.
   private final int retryLongerMultiplier;
+  private final int syncWaitTimeout;
   private boolean aborted;
   private boolean cleanupConnectionOnClose = false; // close the connection in close()
   private boolean closed = false;
@@ -242,6 +253,8 @@ public class HBaseAdmin implements Admin {
         "hbase.client.retries.longer.multiplier", 10);
     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
+    this.syncWaitTimeout = this.conf.getInt(
+      "hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
 
     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
   }
@@ -541,92 +554,23 @@ public class HBaseAdmin implements Admin {
    */
   @Override
   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
-  throws IOException {
+      throws IOException {
+    Future<Void> future = createTableAsyncV2(desc, splitKeys);
     try {
-      createTableAsync(desc, splitKeys);
-    } catch (SocketTimeoutException ste) {
-      LOG.warn("Creating " + desc.getTableName() + " took too long", ste);
-    }
-    int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
-    int prevRegCount = 0;
-    boolean tableWasEnabled = false;
-    for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
-      ++tries) {
-      if (tableWasEnabled) {
-        // Wait all table regions comes online
-        final AtomicInteger actualRegCount = new AtomicInteger(0);
-        MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
-          @Override
-          public boolean visit(Result rowResult) throws IOException {
-            RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
-            if (list == null) {
-              LOG.warn("No serialized HRegionInfo in " + rowResult);
-              return true;
-            }
-            HRegionLocation l = list.getRegionLocation();
-            if (l == null) {
-              return true;
-            }
-            if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
-              return false;
-            }
-            if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
-            HRegionLocation[] locations = list.getRegionLocations();
-            for (HRegionLocation location : locations) {
-              if (location == null) continue;
-              ServerName serverName = location.getServerName();
-              // Make sure that regions are assigned to server
-              if (serverName != null && serverName.getHostAndPort() != null) {
-                actualRegCount.incrementAndGet();
-              }
-            }
-            return true;
-          }
-        };
-        MetaTableAccessor.scanMetaForTableRegions(connection, visitor, desc.getTableName());
-        if (actualRegCount.get() < numRegs) {
-          if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
-            throw new RegionOfflineException("Only " + actualRegCount.get() +
-              " of " + numRegs + " regions are online; retries exhausted.");
-          }
-          try { // Sleep
-            Thread.sleep(getPauseTime(tries));
-          } catch (InterruptedException e) {
-            throw new InterruptedIOException("Interrupted when opening" +
-              " regions; " + actualRegCount.get() + " of " + numRegs +
-              " regions processed so far");
-          }
-          if (actualRegCount.get() > prevRegCount) { // Making progress
-            prevRegCount = actualRegCount.get();
-            tries = -1;
-          }
-        } else {
-          return;
-        }
+      // TODO: how long should we wait? spin forever?
+      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted when waiting" +
+          " for table to be enabled; meta scan was done");
+    } catch (TimeoutException e) {
+      throw new TimeoutIOException(e);
+    } catch (ExecutionException e) {
+      if (e.getCause() instanceof IOException) {
+        throw (IOException)e.getCause();
       } else {
-        try {
-          tableWasEnabled = isTableAvailable(desc.getTableName());
-        } catch (TableNotFoundException tnfe) {
-          LOG.debug(
-              "Table " + desc.getTableName() + " was not enabled, sleeping, still " + numRetries
-                  + " retries left");
-        }
-        if (tableWasEnabled) {
-          // no we will scan meta to ensure all regions are online
-          tries = -1;
-        } else {
-          try { // Sleep
-            Thread.sleep(getPauseTime(tries));
-          } catch (InterruptedException e) {
-            throw new InterruptedIOException("Interrupted when waiting" +
-                " for table to be enabled; meta scan was done");
-          }
-        }
+        throw new IOException(e.getCause());
       }
     }
-    throw new TableNotEnabledException(
-      "Retries exhausted while still waiting for table: "
-      + desc.getTableName() + " to be enabled");
   }
 
   /**
@@ -646,22 +590,42 @@ public class HBaseAdmin implements Admin {
    * @throws IOException
    */
   @Override
-  public void createTableAsync(
-    final HTableDescriptor desc, final byte [][] splitKeys)
-  throws IOException {
-    if(desc.getTableName() == null) {
+  public void createTableAsync(final HTableDescriptor desc, final byte [][] splitKeys)
+      throws IOException {
+    createTableAsyncV2(desc, splitKeys);
+  }
+
+  /**
+   * Creates a new table but does not block and wait for it to come online.
+   * 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 desc table descriptor for table
+   * @param splitKeys keys to check if the table has been created with all split keys
+   * @throws IllegalArgumentException Bad table name, if the split keys
+   *    are repeated and if the split key has empty byte array.
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async creation. 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> createTableAsyncV2(final HTableDescriptor desc, final byte[][] splitKeys)
+      throws IOException {
+    if (desc.getTableName() == null) {
       throw new IllegalArgumentException("TableName cannot be null");
     }
-    if(splitKeys != null && splitKeys.length > 0) {
+    if (splitKeys != null && splitKeys.length > 0) {
       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
       // Verify there are no duplicate split keys
-      byte [] lastKey = null;
-      for(byte [] splitKey : splitKeys) {
+      byte[] lastKey = null;
+      for (byte[] splitKey : splitKeys) {
         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
           throw new IllegalArgumentException(
               "Empty split key must not be passed in the split keys.");
         }
-        if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
+        if (lastKey != null && Bytes.equals(splitKey, lastKey)) {
           throw new IllegalArgumentException("All split keys must be unique, " +
             "found duplicate: " + Bytes.toStringBinary(splitKey) +
             ", " + Bytes.toStringBinary(lastKey));
@@ -670,14 +634,127 @@ public class HBaseAdmin implements Admin {
       }
     }
 
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+    CreateTableResponse response = executeCallable(
+        new MasterCallable<CreateTableResponse>(getConnection()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
+      public CreateTableResponse call(int callTimeout) throws ServiceException {
         CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys);
-        master.createTable(null, request);
-        return null;
+        return master.createTable(null, request);
       }
     });
+    return new CreateTableFuture(this, desc, splitKeys, response);
+  }
+
+  private static class CreateTableFuture extends ProcedureFuture<Void> {
+    private final HTableDescriptor desc;
+    private final byte[][] splitKeys;
+
+    public CreateTableFuture(final HBaseAdmin admin, final HTableDescriptor desc,
+        final byte[][] splitKeys, final CreateTableResponse response) {
+      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
+      this.splitKeys = splitKeys;
+      this.desc = desc;
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForTableEnabled(deadlineTs);
+      waitForAllRegionsOnline(deadlineTs);
+      return null;
+    }
+
+    @Override
+    protected Void postOperationResult(final Void result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      LOG.info("Created " + desc.getTableName());
+      return result;
+    }
+
+    private void waitForTableEnabled(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForState(deadlineTs, new WaitForStateCallable() {
+        @Override
+        public boolean checkState(int tries) throws IOException {
+          try {
+            if (getAdmin().isTableAvailable(desc.getTableName())) {
+              return true;
+            }
+          } catch (TableNotFoundException tnfe) {
+            LOG.debug("Table "+ desc.getTableName() +" was not enabled, sleeping. tries="+  tries);
+          }
+          return false;
+        }
+
+        @Override
+        public void throwInterruptedException() throws InterruptedIOException {
+          throw new InterruptedIOException("Interrupted when waiting for table " +
+              desc.getTableName() + " to be enabled");
+        }
+
+        @Override
+        public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+          throw new TimeoutException("Table " + desc.getTableName() +
+            " not enabled after " + elapsedTime + "msec");
+        }
+      });
+    }
+
+    private void waitForAllRegionsOnline(final long deadlineTs)
+        throws IOException, TimeoutException {
+      final AtomicInteger actualRegCount = new AtomicInteger(0);
+      final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
+        @Override
+        public boolean visit(Result rowResult) throws IOException {
+          RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
+          if (list == null) {
+            LOG.warn("No serialized HRegionInfo in " + rowResult);
+            return true;
+          }
+          HRegionLocation l = list.getRegionLocation();
+          if (l == null) {
+            return true;
+          }
+          if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
+            return false;
+          }
+          if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
+          HRegionLocation[] locations = list.getRegionLocations();
+          for (HRegionLocation location : locations) {
+            if (location == null) continue;
+            ServerName serverName = location.getServerName();
+            // Make sure that regions are assigned to server
+            if (serverName != null && serverName.getHostAndPort() != null) {
+              actualRegCount.incrementAndGet();
+            }
+          }
+          return true;
+        }
+      };
+
+      int tries = 0;
+      IOException serverEx = null;
+      int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
+      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
+        actualRegCount.set(0);
+        MetaTableAccessor.scanMetaForTableRegions(
+          getAdmin().getConnection(), visitor, desc.getTableName());
+        if (actualRegCount.get() == numRegs) {
+          // all the regions are online
+          return;
+        }
+
+        try {
+          Thread.sleep(getAdmin().getPauseTime(tries++));
+        } catch (InterruptedException e) {
+          throw new InterruptedIOException("Interrupted when opening" +
+            " regions; " + actualRegCount.get() + " of " + numRegs +
+            " regions processed so far");
+        }
+      }
+      throw new TimeoutException("Only " + actualRegCount.get() +
+              " of " + numRegs + " regions are online; retries exhausted.");
+    }
   }
 
   public void deleteTable(final String tableName) throws IOException {
@@ -697,48 +774,93 @@ public class HBaseAdmin implements Admin {
    */
   @Override
   public void deleteTable(final TableName tableName) throws IOException {
-    boolean tableExists = true;
+    Future<Void> future = deleteTableAsyncV2(tableName);
+    try {
+      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
+    } 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());
+      }
+    }
+  }
 
-    executeCallable(new MasterCallable<Void>(getConnection()) {
+  /**
+   * Deletes the table but does not block and wait for it be completely removed.
+   * 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 desc table descriptor for table
+   * @param tableName name of table to delete
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async delete. 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> deleteTableAsyncV2(final TableName tableName) throws IOException {
+    DeleteTableResponse response = executeCallable(
+        new MasterCallable<DeleteTableResponse>(getConnection()) {
       @Override
-      public Void call(int callTimeout) throws ServiceException {
+      public DeleteTableResponse call(int callTimeout) throws ServiceException {
         DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName);
-        master.deleteTable(null,req);
-        return null;
+        return master.deleteTable(null,req);
       }
     });
+    return new DeleteTableFuture(this, tableName, response);
+  }
 
-    int failures = 0;
-    for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
-      try {
-        tableExists = tableExists(tableName);
-        if (!tableExists)
-          break;
-      } catch (IOException ex) {
-        failures++;
-        if(failures >= numRetries - 1) {           // no more tries left
-          if (ex instanceof RemoteException) {
-            throw ((RemoteException) ex).unwrapRemoteException();
-          } else {
-            throw ex;
-          }
-        }
-      }
-      try {
-        Thread.sleep(getPauseTime(tries));
-      } catch (InterruptedException e) {
-        throw new InterruptedIOException("Interrupted when waiting" +
-            " for table to be deleted");
-      }
+  private static class DeleteTableFuture extends ProcedureFuture<Void> {
+    private final TableName tableName;
+
+    public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
+        final DeleteTableResponse response) {
+      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
+      this.tableName = tableName;
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitTableNotFound(deadlineTs);
+      return null;
+    }
+
+    @Override
+    protected Void postOperationResult(final Void result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      // Delete cached information to prevent clients from using old locations
+      getAdmin().getConnection().clearRegionCache(tableName);
+      LOG.info("Deleted " + tableName);
+      return result;
     }
 
-    if (tableExists) {
-      throw new IOException("Retries exhausted, it took too long to wait"+
-        " for the table " + tableName + " to be deleted.");
+    private void waitTableNotFound(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForState(deadlineTs, new WaitForStateCallable() {
+        @Override
+        public boolean checkState(int tries) throws IOException {
+          return !getAdmin().tableExists(tableName);
+        }
+
+        @Override
+        public void throwInterruptedException() throws InterruptedIOException {
+          throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
+        }
+
+        @Override
+        public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+          throw new TimeoutException("Table " + tableName + " not yet deleted after " +
+              elapsedTime + "msec");
+        }
+      });
     }
-    // Delete cached information to prevent clients from using old locations
-    this.connection.clearRegionCache(tableName);
-    LOG.info("Deleted " + tableName);
   }
 
   /**
@@ -3834,4 +3956,236 @@ public class HBaseAdmin implements Admin {
       }
     });
   }
+
+  /**
+   * Future that waits on a procedure result.
+   * Returned by the async version of the Admin calls,
+   * and used internally by the sync calls to wait on the result of the procedure.
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  protected static class ProcedureFuture<V> implements Future<V> {
+    private ExecutionException exception = null;
+    private boolean procResultFound = false;
+    private boolean done = false;
+    private V result = null;
+
+    private final HBaseAdmin admin;
+    private final Long procId;
+
+    public ProcedureFuture(final HBaseAdmin admin, final Long procId) {
+      this.admin = admin;
+      this.procId = procId;
+    }
+
+    @Override
+    public boolean cancel(boolean mayInterruptIfRunning) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean isCancelled() {
+      // TODO: Abort not implemented yet
+      return false;
+    }
+
+    @Override
+    public V get() throws InterruptedException, ExecutionException {
+      // TODO: should we ever spin forever?
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public V get(long timeout, TimeUnit unit)
+        throws InterruptedException, ExecutionException, TimeoutException {
+      if (!done) {
+        long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
+        try {
+          try {
+            // if the master support procedures, try to wait the result
+            if (procId != null) {
+              result = waitProcedureResult(procId, deadlineTs);
+            }
+            // if we don't have a proc result, try the compatibility wait
+            if (!procResultFound) {
+              result = waitOperationResult(deadlineTs);
+            }
+            result = postOperationResult(result, deadlineTs);
+            done = true;
+          } catch (IOException e) {
+            result = postOpeartionFailure(e, deadlineTs);
+            done = true;
+          }
+        } catch (IOException e) {
+          exception = new ExecutionException(e);
+          done = true;
+        }
+      }
+      if (exception != null) {
+        throw exception;
+      }
+      return result;
+    }
+
+    @Override
+    public boolean isDone() {
+      return done;
+    }
+
+    protected HBaseAdmin getAdmin() {
+      return admin;
+    }
+
+    private V waitProcedureResult(long procId, long deadlineTs)
+        throws IOException, TimeoutException, InterruptedException {
+      GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder()
+          .setProcId(procId)
+          .build();
+
+      int tries = 0;
+      IOException serviceEx = null;
+      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
+        GetProcedureResultResponse response = null;
+        try {
+          // Try to fetch the result
+          response = getProcedureResult(request);
+        } catch (IOException e) {
+          serviceEx = unwrapException(e);
+
+          // the master may be down
+          LOG.warn("failed to get the procedure result procId=" + procId, serviceEx);
+
+          // Not much to do, if we have a DoNotRetryIOException
+          if (serviceEx instanceof DoNotRetryIOException) {
+            // TODO: looks like there is no way to unwrap this exception and get the proper
+            // UnsupportedOperationException aside from looking at the message.
+            // anyway, if we fail here we just failover to the compatibility side
+            // and that is always a valid solution.
+            LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx);
+            procResultFound = false;
+            return null;
+          }
+        }
+
+        // If the procedure is no longer running, we should have a result
+        if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) {
+          procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND;
+          return convertResult(response);
+        }
+
+        try {
+          Thread.sleep(getAdmin().getPauseTime(tries++));
+        } catch (InterruptedException e) {
+          throw new InterruptedException(
+            "Interrupted while waiting for the result of proc " + procId);
+        }
+      }
+      if (serviceEx != null) {
+        throw serviceEx;
+      } else {
+        throw new TimeoutException("The procedure " + procId + " is still running");
+      }
+    }
+
+    private static IOException unwrapException(IOException e) {
+      if (e instanceof RemoteException) {
+        return ((RemoteException)e).unwrapRemoteException();
+      }
+      return e;
+    }
+
+    protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
+        throws IOException {
+      return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
+          admin.getConnection()) {
+        @Override
+        public GetProcedureResultResponse call(int callTimeout) throws ServiceException {
+          return master.getProcedureResult(null, request);
+        }
+      });
+    }
+
+    /**
+     * Convert the procedure result response to a specified type.
+     * @param response the procedure result object to parse
+     * @return the result data of the procedure.
+     */
+    protected V convertResult(final GetProcedureResultResponse response) throws IOException {
+      if (response.hasException()) {
+        throw ForeignExceptionUtil.toIOException(response.getException());
+      }
+      return null;
+    }
+
+    /**
+     * Fallback implementation in case the procedure is not supported by the server.
+     * It should try to wait until the operation is completed.
+     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
+     * @return the result data of the operation
+     */
+    protected V waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      return null;
+    }
+
+    /**
+     * Called after the operation is completed and the result fetched.
+     * this allows to perform extra steps after the procedure is completed.
+     * it allows to apply transformations to the result that will be returned by get().
+     * @param result the result of the procedure
+     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
+     * @return the result of the procedure, which may be the same as the passed one
+     */
+    protected V postOperationResult(final V result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      return result;
+    }
+
+    /**
+     * Called after the operation is terminated with a failure.
+     * this allows to perform extra steps after the procedure is terminated.
+     * it allows to apply transformations to the result that will be returned by get().
+     * The default implementation will rethrow the exception
+     * @param result the result of the procedure
+     * @param deadlineTs the timestamp after which this method should throw a TimeoutException
+     * @return the result of the procedure, which may be the same as the passed one
+     */
+    protected V postOpeartionFailure(final IOException exception, final long deadlineTs)
+        throws IOException, TimeoutException {
+      throw exception;
+    }
+
+    protected interface WaitForStateCallable {
+      boolean checkState(int tries) throws IOException;
+      void throwInterruptedException() throws InterruptedIOException;
+      void throwTimeoutException(long elapsed) throws TimeoutException;
+    }
+
+    protected void waitForState(final long deadlineTs, final WaitForStateCallable callable)
+        throws IOException, TimeoutException {
+      int tries = 0;
+      IOException serverEx = null;
+      long startTime = EnvironmentEdgeManager.currentTime();
+      while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
+        serverEx = null;
+        try {
+          if (callable.checkState(tries)) {
+            return;
+          }
+        } catch (IOException e) {
+          serverEx = e;
+        }
+        try {
+          Thread.sleep(getAdmin().getPauseTime(tries++));
+        } catch (InterruptedException e) {
+          callable.throwInterruptedException();
+        }
+      }
+      if (serverEx != null) {
+        throw unwrapException(serverEx);
+      } else {
+        callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
new file mode 100644
index 0000000..da3ffe9
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
@@ -0,0 +1,186 @@
+/**
+ *
+ * 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.client;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({ClientTests.class, SmallTests.class})
+public class TestProcedureFuture {
+  private static class TestFuture extends HBaseAdmin.ProcedureFuture<Void> {
+    private boolean postOperationResultCalled = false;
+    private boolean waitOperationResultCalled = false;
+    private boolean getProcedureResultCalled = false;
+    private boolean convertResultCalled = false;
+
+    public TestFuture(final HBaseAdmin admin, final Long procId) {
+      super(admin, procId);
+    }
+
+    public boolean wasPostOperationResultCalled() {
+      return postOperationResultCalled;
+    }
+
+    public boolean wasWaitOperationResultCalled() {
+      return waitOperationResultCalled;
+    }
+
+    public boolean wasGetProcedureResultCalled() {
+      return getProcedureResultCalled;
+    }
+
+    public boolean wasConvertResultCalled() {
+      return convertResultCalled;
+    }
+
+    @Override
+    protected GetProcedureResultResponse getProcedureResult(
+        final GetProcedureResultRequest request) throws IOException {
+      getProcedureResultCalled = true;
+      return GetProcedureResultResponse.newBuilder()
+              .setState(GetProcedureResultResponse.State.FINISHED)
+              .build();
+    }
+
+    @Override
+    protected Void convertResult(final GetProcedureResultResponse response) throws IOException {
+      convertResultCalled = true;
+      return null;
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitOperationResultCalled = true;
+      return null;
+    }
+
+    @Override
+    protected Void postOperationResult(final Void result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      postOperationResultCalled = true;
+      return result;
+    }
+  }
+
+  /**
+   * When a master return a result with procId,
+   * we are skipping the waitOperationResult() call,
+   * since we are getting the procedure result.
+   */
+  @Test(timeout=60000)
+  public void testWithProcId() throws Exception {
+    HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
+    TestFuture f = new TestFuture(admin, 100L);
+    f.get(1, TimeUnit.MINUTES);
+
+    assertTrue("expected getProcedureResult() to be called", f.wasGetProcedureResultCalled());
+    assertTrue("expected convertResult() to be called", f.wasConvertResultCalled());
+    assertFalse("unexpected waitOperationResult() called", f.wasWaitOperationResultCalled());
+    assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled());
+  }
+
+  /**
+   * Verify that the spin loop for the procedure running works.
+   */
+  @Test(timeout=60000)
+  public void testWithProcIdAndSpinning() throws Exception {
+    final AtomicInteger spinCount = new AtomicInteger(0);
+    HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
+    TestFuture f = new TestFuture(admin, 100L) {
+      @Override
+      protected GetProcedureResultResponse getProcedureResult(
+          final GetProcedureResultRequest request) throws IOException {
+        boolean done = spinCount.incrementAndGet() >= 10;
+        return GetProcedureResultResponse.newBuilder()
+              .setState(done ? GetProcedureResultResponse.State.FINISHED :
+                GetProcedureResultResponse.State.RUNNING)
+              .build();
+      }
+    };
+    f.get(1, TimeUnit.MINUTES);
+
+    assertEquals(10, spinCount.get());
+    assertTrue("expected convertResult() to be called", f.wasConvertResultCalled());
+    assertFalse("unexpected waitOperationResult() called", f.wasWaitOperationResultCalled());
+    assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled());
+  }
+
+  /**
+   * When a master return a result without procId,
+   * we are skipping the getProcedureResult() call.
+   */
+  @Test(timeout=60000)
+  public void testWithoutProcId() throws Exception {
+    HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
+    TestFuture f = new TestFuture(admin, null);
+    f.get(1, TimeUnit.MINUTES);
+
+    assertFalse("unexpected getProcedureResult() called", f.wasGetProcedureResultCalled());
+    assertFalse("unexpected convertResult() called", f.wasConvertResultCalled());
+    assertTrue("expected waitOperationResult() to be called", f.wasWaitOperationResultCalled());
+    assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled());
+  }
+
+  /**
+   * When a new client with procedure support tries to ask an old-master without proc-support
+   * the procedure result we get a DoNotRetryIOException (which is an UnsupportedOperationException)
+   * The future should trap that and fallback to the waitOperationResult().
+   *
+   * This happens when the operation calls happens on a "new master" but while we are waiting
+   * the operation to be completed, we failover on an "old master".
+   */
+  @Test(timeout=60000)
+  public void testOnServerWithNoProcedureSupport() throws Exception {
+    HBaseAdmin admin = Mockito.mock(HBaseAdmin.class);
+    TestFuture f = new TestFuture(admin, 100L) {
+      @Override
+      protected GetProcedureResultResponse getProcedureResult(
+        final GetProcedureResultRequest request) throws IOException {
+        super.getProcedureResult(request);
+        throw new DoNotRetryIOException(new UnsupportedOperationException("getProcedureResult"));
+      }
+    };
+    f.get(1, TimeUnit.MINUTES);
+
+    assertTrue("expected getProcedureResult() to be called", f.wasGetProcedureResultCalled());
+    assertFalse("unexpected convertResult() called", f.wasConvertResultCalled());
+    assertTrue("expected waitOperationResult() to be called", f.wasWaitOperationResultCalled());
+    assertTrue("expected postOperationResult() to be called", f.wasPostOperationResultCalled());
+  }
+}
\ No newline at end of file


[13/18] 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/5946a525/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 5c96f21..6f40986 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,65 +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_ModifyTableMessage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableMessage_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 AddColumnFamilyMessageOrBuilder
+      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\"\337\001\n\022ModifyTableMessa" +
-      "ge\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation\022" +
-      "-\n\027unmodified_table_schema\030\002 \001(\0132\014.Table" +
-      "Schema\022+\n\025modified_table_schema\030\003 \002(\0132\014." +
-      "TableSchema\022&\n\036delete_column_family_in_m" +
-      "odify\030\004 \002(\010\022 \n\013region_info\030\005 \003(\0132\013.Regio",
-      "nInfo\"}\n\024DeleteTableStateData\022#\n\tuser_in" +
-      "fo\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_name" +
-      "\030\002 \002(\0132\n.TableName\022 \n\013region_info\030\003 \003(\0132" +
-      "\013.RegionInfo*\330\001\n\020CreateTableState\022\036\n\032CRE" +
-      "ATE_TABLE_PRE_OPERATION\020\001\022 \n\034CREATE_TABL" +
-      "E_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\036CREATE_TABLE_UPDATE_DESC_CACHE\020\005\022\037" +
-      "\n\033CREATE_TABLE_POST_OPERATION\020\006*\207\002\n\020Modi" +
-      "fyTableState\022\030\n\024MODIFY_TABLE_PREPARE\020\001\022\036",
-      "\n\032MODIFY_TABLE_PRE_OPERATION\020\002\022(\n$MODIFY" +
-      "_TABLE_UPDATE_TABLE_DESCRIPTOR\020\003\022&\n\"MODI" +
-      "FY_TABLE_REMOVE_REPLICA_COLUMN\020\004\022!\n\035MODI" +
-      "FY_TABLE_DELETE_FS_LAYOUT\020\005\022\037\n\033MODIFY_TA" +
-      "BLE_POST_OPERATION\020\006\022#\n\037MODIFY_TABLE_REO" +
-      "PEN_ALL_REGIONS\020\007*\337\001\n\020DeleteTableState\022\036" +
-      "\n\032DELETE_TABLE_PRE_OPERATION\020\001\022!\n\035DELETE" +
-      "_TABLE_REMOVE_FROM_META\020\002\022 \n\034DELETE_TABL" +
-      "E_CLEAR_FS_LAYOUT\020\003\022\"\n\036DELETE_TABLE_UPDA" +
-      "TE_DESC_CACHE\020\004\022!\n\035DELETE_TABLE_UNASSIGN",
-      "_REGIONS\020\005\022\037\n\033DELETE_TABLE_POST_OPERATIO" +
-      "N\020\006BK\n*org.apache.hadoop.hbase.protobuf." +
-      "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 AddColumnFamilyMessage}
+   */
+  public static final class AddColumnFamilyMessage extends
+      com.google.protobuf.GeneratedMessage
+      implements AddColumnFamilyMessageOrBuilder {
+    // Use AddColumnFamilyMessage.newBuilder() to construct.
+    private AddColumnFamilyMessage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private AddColumnFamilyMessage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final AddColumnFamilyMessage defaultInstance;
+    public static AddColumnFamilyMessage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public AddColumnFamilyMessage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private AddColumnFamilyMessage(
+        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_AddColumnFamilyMessage_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyMessage_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<AddColumnFamilyMessage> PARSER =
+        new com.google.protobuf.AbstractParser<AddColumnFamilyMessage>() {
+      public AddColumnFamilyMessage parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new AddColumnFamilyMessage(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<AddColumnFamilyMessage> 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.AddColumnFamilyMessage)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage) 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.AddColumnFamilyMessage parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage 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.AddColumnFamilyMessage parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage 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.AddColumnFamilyMessage parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage 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.AddColumnFamilyMessage parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage 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.AddColumnFamilyMessage parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage 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.AddColumnFamilyMessage 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 AddColumnFamilyMessage}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessageOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyMessage_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyMessage_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.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_AddColumnFamilyMessage_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage(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.AddColumnFamilyMessage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage.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.AddColumnFamilyMessage parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyMessage) 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:AddColumnFamilyMessage)
+    }
+
+    static {
+      defaultInstance = new AddColumnFamilyMessage(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:AddColumnFamilyMessage)
+  }
+
+  public interface ModifyColumnFamilyMessageOrBuilder
+      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 ModifyColumnFamilyMessage}
+   */
+  public static final class ModifyColumnFamilyMessage extends
+      com.google.protobuf.GeneratedMessage
+      implements ModifyColumnFamilyMessageOrBuilder {
+    // Use ModifyColumnFamilyMessage.newBuilder() to construct.
+    private ModifyColumnFamilyMessage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ModifyColumnFamilyMessage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ModifyColumnFamilyMessage defaultInstance;
+    public static ModifyColumnFamilyMessage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ModifyColumnFamilyMessage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ModifyColumnFamilyMessage(
+        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.ap

<TRUNCATED>

[09/18] hbase git commit: HBASE-13204 Procedure v2 - client create/delete table sync

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/02c15a2f/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 dce0737..cc6f201 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
@@ -9074,6 +9074,16 @@ public final class MasterProtos {
 
   public interface CreateTableResponseOrBuilder
       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 CreateTableResponse}
@@ -9108,6 +9118,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 {
@@ -9125,6 +9136,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -9164,7 +9180,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() {
@@ -9178,6 +9212,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);
     }
 
@@ -9187,6 +9224,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;
@@ -9210,6 +9251,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -9223,6 +9269,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;
@@ -9332,6 +9382,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -9358,6 +9410,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse(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;
       }
@@ -9373,6 +9432,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9398,6 +9460,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:CreateTableResponse)
     }
@@ -9973,6 +10069,16 @@ public final class MasterProtos {
 
   public interface DeleteTableResponseOrBuilder
       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 DeleteTableResponse}
@@ -10007,6 +10113,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 {
@@ -10024,6 +10131,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -10063,7 +10175,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() {
@@ -10077,6 +10207,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);
     }
 
@@ -10086,6 +10219,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;
@@ -10109,6 +10246,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -10122,6 +10264,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;
@@ -10231,6 +10377,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -10257,6 +10405,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse(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;
       }
@@ -10272,6 +10427,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -10297,6 +10455,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:DeleteTableResponse)
     }
@@ -42809,21 +43001,1161 @@ public final class MasterProtos {
               break;
             }
             case 8: {
-              bitField0_ |= 0x00000001;
-              done_ = input.readBool();
+              bitField0_ |= 0x00000001;
+              done_ = input.readBool();
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = snapshot_.toBuilder();
+              }
+              snapshot_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(snapshot_);
+                snapshot_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              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.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<IsProcedureDoneResponse> PARSER =
+        new com.google.protobuf.AbstractParser<IsProcedureDoneResponse>() {
+      public IsProcedureDoneResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new IsProcedureDoneResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<IsProcedureDoneResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional bool done = 1 [default = false];
+    public static final int DONE_FIELD_NUMBER = 1;
+    private boolean done_;
+    /**
+     * <code>optional bool done = 1 [default = false];</code>
+     */
+    public boolean hasDone() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional bool done = 1 [default = false];</code>
+     */
+    public boolean getDone() {
+      return done_;
+    }
+
+    // optional .ProcedureDescription snapshot = 2;
+    public static final int SNAPSHOT_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription snapshot_;
+    /**
+     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     */
+    public boolean hasSnapshot() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription getSnapshot() {
+      return snapshot_;
+    }
+    /**
+     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder getSnapshotOrBuilder() {
+      return snapshot_;
+    }
+
+    private void initFields() {
+      done_ = false;
+      snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (hasSnapshot()) {
+        if (!getSnapshot().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.writeBool(1, done_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, snapshot_);
+      }
+      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
+          .computeBoolSize(1, done_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, snapshot_);
+      }
+      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.MasterProtos.IsProcedureDoneResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse) obj;
+
+      boolean result = true;
+      result = result && (hasDone() == other.hasDone());
+      if (hasDone()) {
+        result = result && (getDone()
+            == other.getDone());
+      }
+      result = result && (hasSnapshot() == other.hasSnapshot());
+      if (hasSnapshot()) {
+        result = result && getSnapshot()
+            .equals(other.getSnapshot());
+      }
+      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 (hasDone()) {
+        hash = (37 * hash) + DONE_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getDone());
+      }
+      if (hasSnapshot()) {
+        hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
+        hash = (53 * hash) + getSnapshot().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse 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.MasterProtos.IsProcedureDoneResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse 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.MasterProtos.IsProcedureDoneResponse 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 IsProcedureDoneResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getSnapshotFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        done_ = false;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (snapshotBuilder_ == null) {
+          snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+        } else {
+          snapshotBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.done_ = done_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (snapshotBuilder_ == null) {
+          result.snapshot_ = snapshot_;
+        } else {
+          result.snapshot_ = snapshotBuilder_.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.MasterProtos.IsProcedureDoneResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance()) return this;
+        if (other.hasDone()) {
+          setDone(other.getDone());
+        }
+        if (other.hasSnapshot()) {
+          mergeSnapshot(other.getSnapshot());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasSnapshot()) {
+          if (!getSnapshot().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.MasterProtos.IsProcedureDoneResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional bool done = 1 [default = false];
+      private boolean done_ ;
+      /**
+       * <code>optional bool done = 1 [default = false];</code>
+       */
+      public boolean hasDone() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional bool done = 1 [default = false];</code>
+       */
+      public boolean getDone() {
+        return done_;
+      }
+      /**
+       * <code>optional bool done = 1 [default = false];</code>
+       */
+      public Builder setDone(boolean value) {
+        bitField0_ |= 0x00000001;
+        done_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool done = 1 [default = false];</code>
+       */
+      public Builder clearDone() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        done_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional .ProcedureDescription snapshot = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> snapshotBuilder_;
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public boolean hasSnapshot() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription getSnapshot() {
+        if (snapshotBuilder_ == null) {
+          return snapshot_;
+        } else {
+          return snapshotBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public Builder setSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription value) {
+        if (snapshotBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          snapshot_ = value;
+          onChanged();
+        } else {
+          snapshotBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public Builder setSnapshot(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder builderForValue) {
+        if (snapshotBuilder_ == null) {
+          snapshot_ = builderForValue.build();
+          onChanged();
+        } else {
+          snapshotBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public Builder mergeSnapshot(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription value) {
+        if (snapshotBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              snapshot_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance()) {
+            snapshot_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.newBuilder(snapshot_).mergeFrom(value).buildPartial();
+          } else {
+            snapshot_ = value;
+          }
+          onChanged();
+        } else {
+          snapshotBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public Builder clearSnapshot() {
+        if (snapshotBuilder_ == null) {
+          snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+          onChanged();
+        } else {
+          snapshotBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder getSnapshotBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getSnapshotFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder getSnapshotOrBuilder() {
+        if (snapshotBuilder_ != null) {
+          return snapshotBuilder_.getMessageOrBuilder();
+        } else {
+          return snapshot_;
+        }
+      }
+      /**
+       * <code>optional .ProcedureDescription snapshot = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder> 
+          getSnapshotFieldBuilder() {
+        if (snapshotBuilder_ == null) {
+          snapshotBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder>(
+                  snapshot_,
+                  getParentForChildren(),
+                  isClean());
+          snapshot_ = null;
+        }
+        return snapshotBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:IsProcedureDoneResponse)
+    }
+
+    static {
+      defaultInstance = new IsProcedureDoneResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:IsProcedureDoneResponse)
+  }
+
+  public interface GetProcedureResultRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required uint64 proc_id = 1;
+    /**
+     * <code>required uint64 proc_id = 1;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>required uint64 proc_id = 1;</code>
+     */
+    long getProcId();
+  }
+  /**
+   * Protobuf type {@code GetProcedureResultRequest}
+   */
+  public static final class GetProcedureResultRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements GetProcedureResultRequestOrBuilder {
+    // Use GetProcedureResultRequest.newBuilder() to construct.
+    private GetProcedureResultRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetProcedureResultRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetProcedureResultRequest defaultInstance;
+    public static GetProcedureResultRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetProcedureResultRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetProcedureResultRequest(
+        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 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              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.MasterProtos.internal_static_GetProcedureResultRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<GetProcedureResultRequest> PARSER =
+        new com.google.protobuf.AbstractParser<GetProcedureResultRequest>() {
+      public GetProcedureResultRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new GetProcedureResultRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<GetProcedureResultRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required uint64 proc_id = 1;
+    public static final int PROC_ID_FIELD_NUMBER = 1;
+    private long procId_;
+    /**
+     * <code>required uint64 proc_id = 1;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required uint64 proc_id = 1;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
+    private void initFields() {
+      procId_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasProcId()) {
+        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.writeUInt64(1, procId_);
+      }
+      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
+          .computeUInt64Size(1, procId_);
+      }
+      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.MasterProtos.GetProcedureResultRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest) obj;
+
+      boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
+      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 (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest 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.MasterProtos.GetProcedureResultRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest 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.MasterProtos.GetProcedureResultRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest 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.MasterProtos.GetProcedureResultRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest 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.MasterProtos.GetProcedureResultRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest 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.MasterProtos.GetProcedureResultRequest 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 GetProcedureResultRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest(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;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasProcId()) {
+          
+          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.MasterProtos.GetProcedureResultRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required uint64 proc_id = 1;
+      private long procId_ ;
+      /**
+       * <code>required uint64 proc_id = 1;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required uint64 proc_id = 1;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>required uint64 proc_id = 1;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000001;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 proc_id = 1;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:GetProcedureResultRequest)
+    }
+
+    static {
+      defaultInstance = new GetProcedureResultRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:GetProcedureResultRequest)
+  }
+
+  public interface GetProcedureResultResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .GetProcedureResultResponse.State state = 1;
+    /**
+     * <code>required .GetProcedureResultResponse.State state = 1;</code>
+     */
+    boolean hasState();
+    /**
+     * <code>required .GetProcedureResultResponse.State state = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State getState();
+
+    // optional uint64 start_time = 2;
+    /**
+     * <code>optional uint64 start_time = 2;</code>
+     */
+    boolean hasStartTime();
+    /**
+     * <code>optional uint64 start_time = 2;</code>
+     */
+    long getStartTime();
+
+    // optional uint64 last_update = 3;
+    /**
+     * <code>optional uint64 last_update = 3;</code>
+     */
+    boolean hasLastUpdate();
+    /**
+     * <code>optional uint64 last_update = 3;</code>
+     */
+    long getLastUpdate();
+
+    // optional bytes result = 4;
+    /**
+     * <code>optional bytes result = 4;</code>
+     */
+    boolean hasResult();
+    /**
+     * <code>optional bytes result = 4;</code>
+     */
+    com.google.protobuf.ByteString getResult();
+
+    // optional .ForeignExceptionMessage exception = 5;
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    boolean hasException();
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage getException();
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder getExceptionOrBuilder();
+  }
+  /**
+   * Protobuf type {@code GetProcedureResultResponse}
+   */
+  public static final class GetProcedureResultResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements GetProcedureResultResponseOrBuilder {
+    // Use GetProcedureResultResponse.newBuilder() to construct.
+    private GetProcedureResultResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private GetProcedureResultResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final GetProcedureResultResponse defaultInstance;
+    public static GetProcedureResultResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public GetProcedureResultResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private GetProcedureResultResponse(
+        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 8: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State value = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                state_ = value;
+              }
               break;
             }
-            case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = snapshot_.toBuilder();
+            case 16: {
+              bitField0_ |= 0x00000002;
+              startTime_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              lastUpdate_ = input.readUInt64();
+              break;
+            }
+            case 34: {
+              bitField0_ |= 0x00000008;
+              result_ = input.readBytes();
+              break;
+            }
+            case 42: {
+              org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000010) == 0x00000010)) {
+                subBuilder = exception_.toBuilder();
               }
-              snapshot_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.PARSER, extensionRegistry);
+              exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(snapshot_);
-                snapshot_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(exception_);
+                exception_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00000002;
+              bitField0_ |= 0x00000010;
               break;
             }
           }
@@ -42840,84 +44172,224 @@ public final class MasterProtos {
     }
     public static final com.google.protobuf.Descriptors.Descriptor
         getDescriptor() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultResponse_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultResponse_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<IsProcedureDoneResponse> PARSER =
-        new com.google.protobuf.AbstractParser<IsProcedureDoneResponse>() {
-      public IsProcedureDoneResponse parsePartialFrom(
+    public static com.google.protobuf.Parser<GetProcedureResultResponse> PARSER =
+        new com.google.protobuf.AbstractParser<GetProcedureResultResponse>() {
+      public GetProcedureResultResponse parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new IsProcedureDoneResponse(input, extensionRegistry);
+        return new GetProcedureResultResponse(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<IsProcedureDoneResponse> getParserForType() {
+    public com.google.protobuf.Parser<GetProcedureResultResponse> getParserForType() {
       return PARSER;
     }
 
+    /**
+     * Protobuf enum {@code GetProcedureResultResponse.State}
+     */
+    public enum State
+        implements com.google.protobuf.ProtocolMessageEnum {
+      /**
+       * <code>NOT_FOUND = 0;</code>
+       */
+      NOT_FOUND(0, 0),
+      /**
+       * <code>RUNNING = 1;</code>
+       */
+      RUNNING(1, 1),
+      /**
+       * <code>FINISHED = 2;</code>
+       */
+      FINISHED(2, 2),
+      ;
+
+      /**
+       * <code>NOT_FOUND = 0;</code>
+       */
+      public static final int NOT_FOUND_VALUE = 0;
+      /**
+       * <code>RUNNING = 1;</code>
+       */
+      public static final int RUNNING_VALUE = 1;
+      /**
+       * <code>FINISHED = 2;</code>
+       */
+      public static final int FINISHED_VALUE = 2;
+
+
+      public final int getNumber() { return value; }
+
+      public static State valueOf(int value) {
+        switch (value) {
+          case 0: return NOT_FOUND;
+          case 1: return RUNNING;
+          case 2: return FINISHED;
+          default: return null;
+        }
+      }
+
+      public static com.google.protobuf.Internal.EnumLiteMap<State>
+          internalGetValueMap() {
+        return internalValueMap;
+      }
+      private static com.google.protobuf.Internal.EnumLiteMap<State>
+          internalValueMap =
+            new com.google.protobuf.Internal.EnumLiteMap<State>() {
+              public State findValueByNumber(int number) {
+                return State.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.MasterProtos.GetProcedureResultResponse.getDescriptor().getEnumTypes().get(0);
+      }
+
+      private static final State[] VALUES = values();
+
+      public static State 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 State(int index, int value) {
+        this.index = index;
+        this.value = value;
+      }
+
+      // @@protoc_insertion_point(enum_scope:GetProcedureResultResponse.State)
+    }
+
     private int bitField0_;
-    // optional bool done = 1 [default = false];
-    public static final int DONE_FIELD_NUMBER = 1;
-    private boolean done_;
+    // required .GetProcedureResultResponse.State state = 1;
+    public static final int STATE_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State state_;
     /**
-     * <code>optional bool done = 1 [default = false];</code>
+     * <code>required .GetProcedureResultResponse.State state = 1;</code>
      */
-    public boolean hasDone() {
+    public boolean hasState() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional bool done = 1 [default = false];</code>
+     * <code>required .GetProcedureResultResponse.State state = 1;</code>
      */
-    public boolean getDone() {
-      return done_;
+    public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State getState() {
+      return state_;
     }
 
-    // optional .ProcedureDescription snapshot = 2;
-    public static final int SNAPSHOT_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription snapshot_;
+    // optional uint64 start_time = 2;
+    public static final int START_TIME_FIELD_NUMBER = 2;
+    private long startTime_;
     /**
-     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     * <code>optional uint64 start_time = 2;</code>
      */
-    public boolean hasSnapshot() {
+    public boolean hasStartTime() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     * <code>optional uint64 start_time = 2;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription getSnapshot() {
-      return snapshot_;
+    public long getStartTime() {
+      return startTime_;
     }
+
+    // optional uint64 last_update = 3;
+    public static final int LAST_UPDATE_FIELD_NUMBER = 3;
+    private long lastUpdate_;
     /**
-     * <code>optional .ProcedureDescription snapshot = 2;</code>
+     * <code>optional uint64 last_update = 3;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescriptionOrBuilder getSnapshotOrBuilder() {
-      return snapshot_;
+    public boolean hasLastUpdate() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional uint64 last_update = 3;</code>
+     */
+    public long getLastUpdate() {
+      return lastUpdate_;
+    }
+
+    // optional bytes result = 4;
+    public static final int RESULT_FIELD_NUMBER = 4;
+    private com.google.protobuf.ByteString result_;
+    /**
+     * <code>optional bytes result = 4;</code>
+     */
+    public boolean hasResult() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional bytes result = 4;</code>
+     */
+    public com.google.protobuf.ByteString getResult() {
+      return result_;
+    }
+
+    // optional .ForeignExceptionMessage exception = 5;
+    public static final int EXCEPTION_FIELD_NUMBER = 5;
+    private org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage exception_;
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    public boolean hasException() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage getException() {
+      return exception_;
+    }
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 5;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder getExceptionOrBuilder() {
+      return exception_;
     }
 
     private void initFields() {
-      done_ = false;
-      snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+      state_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State.NOT_FOUND;
+      startTime_ = 0L;
+      lastUpdate_ = 0L;
+      result_ = com.google.protobuf.ByteString.EMPTY;
+      exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
-      if (hasSnapshot()) {
-        if (!getSnapshot().isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
+      if (!hasState()) {
+        memoizedIsInitialized = 0;
+        return false;
       }
       memoizedIsInitialized = 1;
       return true;
@@ -42927,10 +44399,19 @@ public final class MasterProtos {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBool(1, done_);
+        output.writeEnum(1, state_.getNumber());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, snapshot_);
+        output.writeUInt64(2, startTime_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBytes(4, result_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeMessage(5, exception_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -42943,11 +44424,23 @@ public final class MasterProtos {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(1, done_);
+          .computeEnumSize(1, state_.getNumber());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, snapshot_);
+          .computeUInt64Size(2, startTime_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(4, result_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, exception_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -42966,21 +44459,36 @@ public final class MasterProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse) obj;
 
       boolean result = true;
-      result = result && (hasDone() == other.hasDone());
-      if (hasDone()) {
-        result = result && (getDone()
-            == other.getDone());
-      }
-      result = result && (hasSnapshot() == other.hasSnapshot());
-      if (hasSnapshot()) {
-        result = result && getSnapshot()
-            .equals(other.getSnapshot());
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result &&
+            (getState() == other.getState());
+      }
+      result = result && (hasStartTime() == other.hasStartTime());
+      if (hasStartTime()) {
+        result = result && (getStartTime()
+            == other.getStartTime());
+      }
+      result = result && (hasLastUpdate() == other.hasLastUpdate());
+      if (hasLastUpdate()) {
+        result = result && (getLastUpdate()
+            == other.getLastUpdate());
+      }
+      result = result && (hasResult() == other.hasResult());
+      if (hasResult()) {
+        result = result && getResult()
+            .equals(other.getResult());
+      }
+      result = result && (hasException() == other.hasException());
+      if (hasException()) {
+        result = result && getException()
+            .equals(other.getException());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -42995,66 +44503,78 @@ public final class MasterProtos {
       }
       int hash = 41;
       hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasDone()) {
-        hash = (37 * hash) + DONE_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getDone());
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getState());
       }
-      if (hasSnapshot()) {
-        hash = (37 * hash) + SNAPSHOT_FIELD_NUMBER;
-        hash = (53 * hash) + getSnapshot().hashCode();
+      if (hasStartTime()) {
+        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getStartTime());
+      }
+      if (hasLastUpdate()) {
+        hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLastUpdate());
+      }
+      if (hasResult()) {
+        hash = (37 * hash) + RESULT_FIELD_NUMBER;
+        hash = (53 * hash) + getResult().hashCode();
+      }
+      if (hasException()) {
+        hash = (37 * hash) + EXCEPTION_FIELD_NUMBER;
+        hash = (53 * hash) + getException().hashCode();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse 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.MasterProtos.IsProcedureDoneResponse parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse 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.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -43063,7 +44583,7 @@ public final class MasterProtos {
 
     public static Builder newBuilder() { return Builder.create(); }
     public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -43075,24 +44595,24 @@ public final class MasterProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code IsProcedureDoneResponse}
+     * Protobuf type {@code GetProcedureResultResponse}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponseOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponseOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultResponse_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultResponse_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -43104,7 +44624,7 @@ public final class MasterProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getSnapshotFieldBuilder();
+          getExceptionFieldBuilder();
         }
       }
       private static Builder create() {
@@ -43113,14 +44633,20 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
-        done_ = false;
+        state_ = org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.State.NOT_FOUND;
         bitField0_ = (bitField0_ & ~0x00000001);
-        if (snapshotBuilder_ == null) {
-          snapshot_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription.getDefaultInstance();
+        startTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        lastUpdate_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        result_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        if (exceptionBuilder_ == null) {
+          exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
         } else {
-          snapshotBuilder_.clear();
+          exceptionBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000002);
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -43130,36 +44656,48 @@ public final class MasterProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_IsProcedureDoneResponse_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetProcedureResultResponse_descriptor;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.done_ = done_;
+        result.state_ = state_;
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        if (snapshotBuilder_ == null) {
-          result.snapshot_ = snapshot_;
+        result.startTime_ = startTime_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.lastUpdate_ = lastUpdate_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.result_ = result_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        if (exceptionBuilder_ == null) {
+          result.exception_ = exception_;
         } else {
-          result.snapshot_ = snapshotBuilder_.build();
+          result.exception_ = exceptionBuilder_.build();
         }
         result.bitField0_ = to_bitField0_;
         onBuilt();
@@ -43167,32 +44705,39 @@ public final class MasterProtos {
       }
 
       public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse)other);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse)other);
         } else {
           super.mergeFrom(other);
           return this;
         }
       }
 
-      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse other) {
-        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse.getDefaultInstance()) return this;
-        if (other.hasDone()) {
-          setDone(other.getDone());
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse.getDefaultInstance()) return this;
+        if (other.hasState()) {
+          setState(other.getState());
         }
-        if (other.hasSnapshot()) {
-          mergeSnapshot(other.getSnapshot());
+        if (other.hasStartTime()) {
+          setStartTime(other.getStartTime());
+        }
+        if (other.hasLastUpdate()) {
+          setLastUpdate(other.getLastUpdate());
+        }
+        if (other.hasResult()) {
+          setResult(other.getResult());
+        }
+        if (other.hasException()) {
+          mergeException(other.getException());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
-        if (hasSnapshot()) {
-          if (!getSnapshot().isInitialized()) {
-            
-            return false;
-          }
+        if (!hasState()) {
+          
+          return false;
         }
         return true;
       }
@@ -43201,11 +44746,11 @@ public final class MasterProtos {
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws java.io.IOException {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse parsedMessage = null;
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse parsedMessage = null;
         try {
           parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse) e.getUnfinishedMessage();
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.G

<TRUNCATED>

[16/18] 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/6fbf41dd/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 6f40986..503533e 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,121 +9376,1901 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:DeleteColumnFamilyMessage)
   }
 
-  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_ModifyTableMessage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableMessage_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_AddColumnFamilyMessage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_AddColumnFamilyMessage_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyColumnFamilyMessage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyColumnFamilyMessage_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteColumnFamilyMessage_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteColumnFamilyMessage_fieldAccessorTable;
+  public interface EnableTableMessageOrBuilder
+      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\"\337\001\n\022ModifyTableMessa" +
-      "ge\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation\022" +
-      "-\n\027unmodified_table_schema\030\002 \001(\0132\014.Table" +
-      "Schema\022+\n\025modified_table_schema\030\003 \002(\0132\014." +
-      "TableSchema\022&\n\036delete_column_family_in_m" +
-      "odify\030\004 \002(\010\022 \n\013region_info\030\005 \003(\0132\013.Regio",
-      "nInfo\"}\n\024DeleteTableStateData\022#\n\tuser_in" +
-      "fo\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_name" +
-      "\030\002 \002(\0132\n.TableName\022 \n\013region_info\030\003 \003(\0132" +
-      "\013.RegionInfo\"\340\001\n\026AddColumnFamilyMessage\022" +
-      "#\n\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\n" +
-      "table_name\030\002 \002(\0132\n.TableName\0220\n\023columnfa" +
-      "mily_schema\030\003 \002(\0132\023.ColumnFamilySchema\022-" +
-      "\n\027unmodified_table_schema\030\004 \001(\0132\014.TableS" +
-      "chema\022 \n\013region_info\030\005 \003(\0132\013.RegionInfo\"" +
-      "\343\001\n\031ModifyColumnFamilyMessage\022#\n\tuser_in",
-      "fo\030\001 \002(\0132\020.UserInformation\022\036\n\ntable_name" +
-      "\030\002 \002(\0132\n.TableName\0220\n\023columnfamily_schem" +
-      "a\030\003 \002(\0132\023.ColumnFamilySchema\022-\n\027unmodifi" +
-      "ed_table_schema\030\004 \001(\0132\014.TableSchema\022 \n\013r" +
-      "egion_info\030\005 \003(\0132\013.RegionInfo\"\314\001\n\031Delete" +
-      "ColumnFamilyMessage\022#\n\tuser_info\030\001 \002(\0132\020" +
-      ".UserInformation\022\036\n\ntable_name\030\002 \002(\0132\n.T" +
-      "ableName\022\031\n\021columnfamily_name\030\003 \002(\014\022-\n\027u" +
-      "nmodified_table_schema\030\004 \001(\0132\014.TableSche" +
-      "ma\022 \n\013region_info\030\005 \003(\0132\013.RegionInfo*\330\001\n",
-      "\020CreateTableState\022\036\n\032CREATE_TABLE_PRE_OP" +
-      "ERATION\020\001\022 \n\034CREATE_TABLE_WRITE_FS_LAYOU" +
-      "T\020\002\022\034\n\030CREATE_TABLE_ADD_TO_META\020\003\022\037\n\033CRE" +
-      "ATE_TABLE_ASSIGN_REGIONS\020\004\022\"\n\036CREATE_TAB" +
-      "LE_UPDATE_DESC_CACHE\020\005\022\037\n\033CREATE_TABLE_P" +
-      "OST_OPERATION\020\006*\207\002\n\020ModifyTableState\022\030\n\024" +
-      "MODIFY_TABLE_PREPARE\020\001\022\036\n\032MODIFY_TABLE_P" +
-      "RE_OPERATION\020\002\022(\n$MODIFY_TABLE_UPDATE_TA" +
-      "BLE_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_OPERATI" +
-      "ON\020\006\022#\n\037MODIFY_TABLE_REOPEN_ALL_REGIONS\020" +
-      "\007*\337\001\n\020DeleteTableState\022\036\n\032DELETE_TABLE_P" +
-      "RE_OPERATION\020\001\022!\n\035DELETE_TABLE_REMOVE_FR" +
-      "OM_META\020\002\022 \n\034DELETE_TABLE_CLEAR_FS_LAYOU" +
-      "T\020\003\022\"\n\036DELETE_TABLE_UPDATE_DESC_CACHE\020\004\022" +
-      "!\n\035DELETE_TABLE_UNASSIGN_REGIONS\020\005\022\037\n\033DE" +
-      "LETE_TABLE_POST_OPERATION\020\006*\331\001\n\024AddColum" +
-      "nFamilyState\022\035\n\031ADD_COLUMN_FAMILY_PREPAR" +
-      "E\020\001\022#\n\037ADD_COLUMN_FAMILY_PRE_OPERATION\020\002",
-      "\022-\n)ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCR" +
-      "IPTOR\020\003\022$\n ADD_COLUMN_FAMILY_POST_OPERAT" +
-      "ION\020\004\022(\n$ADD_COLUMN_FAMILY_REOPEN_ALL_RE" +
-      "GIONS\020\005*\353\001\n\027ModifyColumnFamilyState\022 \n\034M" +
-      "ODIFY_COLUMN_FAMILY_PREPARE\020\001\022&\n\"MODIFY_" +
-      "COLUMN_FAMILY_PRE_OPERATION\020\002\0220\n,MODIFY_" +
-      "COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022" +
-      "\'\n#MODIFY_COLUMN_FAMILY_POST_OPERATION\020\004" +
-      "\022+\n\'MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGI" +
-      "ONS\020\005*\226\002\n\027DeleteColumnFamilyState\022 \n\034DEL",
-      "ETE_COLUMN_FAMILY_PREPARE\020\001\022&\n\"DELETE_CO" +
-      "LUMN_FAMILY_PRE_OPERATION\020\002\0220\n,DELETE_CO" +
-      "LUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022)\n" +
-      "%DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT\020\004" +
-      "\022\'\n#DELETE_COLUMN_FAMILY_POST_OPERATION\020" +
-      "\005\022+\n\'DELETE_COLUMN_FAMILY_REOPEN_ALL_REG" +
-      "IONS\020\006BK\n*org.apache.hadoop.hbase.protob" +
-      "uf.generatedB\025MasterProcedureProtosH\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;
-          internal_static_CreateTableStateData_descriptor =
+  /**
+   * Protobuf type {@code EnableTableMessage}
+   */
+  public static final class EnableTableMessage extends
+      com.google.protobuf.GeneratedMessage
+      implements EnableTableMessageOrBuilder {
+    // Use EnableTableMessage.newBuilder() to construct.
+    private EnableTableMessage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private EnableTableMessage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final EnableTableMessage defaultInstance;
+    public static EnableTableMessage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public EnableTableMessage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private EnableTableMessage(
+        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_EnableTableMessage_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableMessage_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<EnableTableMessage> PARSER =
+        new com.google.protobuf.AbstractParser<EnableTableMessage>() {
+      public EnableTableMessage parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new EnableTableMessage(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<EnableTableMessage> 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.EnableTableMessage)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage) 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.EnableTableMessage parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage 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.EnableTableMessage parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage 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.EnableTableMessage parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage 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.EnableTableMessage parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage 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.EnableTableMessage parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage 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.EnableTableMessage 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 EnableTableMessage}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessageOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableMessage_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableMessage_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.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_EnableTableMessage_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage(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.EnableTableMessage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage.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.EnableTableMessage parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableMessage) 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:EnableTableMessage)
+    }
+
+    static {
+      defaultInstance = new EnableTableMessage(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:EnableTableMessage)
+  }
+
+  public interface DisableTableMessageOrBuilder
+      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 DisableTableMessage}
+   */
+  public static final class DisableTableMessage extends
+      com.google.protobuf.GeneratedMessage
+      implements DisableTableMessageOrBuilder {
+    // Use DisableTableMessage.newBuilder() to construct.
+    private DisableTableMessage(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private DisableTableMessage(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final DisableTableMessage defaultInstance;
+    public static DisableTableMessage getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public DisableTableMessage getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private DisableTableMessage(
+        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_DisableTableMessage_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableMessage_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<DisableTableMessage> PARSER =
+        new com.google.protobuf.AbstractParser<DisableTableMessage>() {
+      public DisableTableMessage parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new DisableTableMessage(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<DisableTableMessage> 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.DisableTableMessage)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage) 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.DisableTableMessage parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage 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.DisableTableMessage parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage 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.DisableTableMessage parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage 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.DisableTableMessage parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage 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.DisableTableMessage parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage 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.DisableTableMessage 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 DisableTableMessage}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessageOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableMessage_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableMessage_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.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_DisableTableMessage_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage(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.DisableTableMessage) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage.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.DisableTableMessage parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableMessage) 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:DisableTableMessage)
+    }
+
+    static {
+      defaultInstance = new DisableTableMessage(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:DisableTableMessage)
+  }
+
+  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_ModifyTableMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyTableMessage_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_AddColumnFamilyMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_AddColumnFamilyMessage_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyColumnFamilyMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyColumnFamilyMessage_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteColumnFamilyMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteColumnFamilyMessage_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EnableTableMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EnableTableMessage_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DisableTableMessage_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DisableTableMessage_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" +
+      "_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\"\337\001\n\022ModifyTableMessa" +
+ 

<TRUNCATED>