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

[01/28] hbase git commit: HBASE-13275 Setting hbase.security.authorization to false does not disable authorization

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 455bedad5 -> 43f335135 (forced update)


http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
new file mode 100644
index 0000000..7c2cb28
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestWithDisabledAuthorization.java
@@ -0,0 +1,1079 @@
+/*
+ * 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.security.access;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+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.Cell;
+import org.apache.hadoop.hbase.Coprocessor;
+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.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+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.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.TestTableName;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Lists;
+
+@Category({SecurityTests.class, LargeTests.class})
+public class TestWithDisabledAuthorization extends SecureTestUtil {
+  private static final Log LOG = LogFactory.getLog(TestWithDisabledAuthorization.class);
+
+  static {
+    Logger.getLogger(AccessController.class).setLevel(Level.TRACE);
+    Logger.getLogger(AccessControlFilter.class).setLevel(Level.TRACE);
+    Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
+  }
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
+  private static final byte[] TEST_FAMILY2 = Bytes.toBytes("f2");
+  private static final byte[] TEST_ROW = Bytes.toBytes("testrow");
+  private static final byte[] TEST_Q1 = Bytes.toBytes("q1");
+  private static final byte[] TEST_Q2 = Bytes.toBytes("q2");
+  private static final byte[] TEST_Q3 = Bytes.toBytes("q3");
+  private static final byte[] TEST_Q4 = Bytes.toBytes("q4");
+  private static final byte[] ZERO = Bytes.toBytes(0L);
+
+  private static MasterCoprocessorEnvironment CP_ENV;
+  private static AccessController ACCESS_CONTROLLER;
+  private static RegionServerCoprocessorEnvironment RSCP_ENV;
+  private RegionCoprocessorEnvironment RCP_ENV;
+
+  @Rule public TestTableName TEST_TABLE = new TestTableName();
+
+  // default users
+  
+  // superuser
+  private static User SUPERUSER;
+  // user granted with all global permission
+  private static User USER_ADMIN;
+  // user with rw permissions on column family.
+  private static User USER_RW;
+  // user with read-only permissions
+  private static User USER_RO;
+  // user is table owner. will have all permissions on table
+  private static User USER_OWNER;
+  // user with create table permissions alone
+  private static User USER_CREATE;
+  // user with no permissions
+  private static User USER_NONE;
+  // user with only partial read-write perms (on family:q1 only)
+  private static User USER_QUAL;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // Enable security
+    enableSecurity(conf);
+    // We expect 0.98 cell ACL semantics
+    conf.setBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, false);
+    // Enable EXEC permission checking
+    conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
+    // Verify enableSecurity sets up what we require
+    verifyConfiguration(conf);
+
+    // Now, DISABLE only active authorization
+    conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, false);
+
+    // Start the minicluster
+    TEST_UTIL.startMiniCluster();
+    MasterCoprocessorHost cpHost =
+        TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
+    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
+    ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
+    CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+      Coprocessor.PRIORITY_HIGHEST, 1, conf);
+    RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
+      .getRegionServerCoprocessorHost();
+    RSCP_ENV = rsHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+      Coprocessor.PRIORITY_HIGHEST, 1, conf);
+
+    // Wait for the ACL table to become available
+    TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
+
+    // create a set of test users
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
+    USER_ADMIN = User.createUserForTesting(conf, "admin2", new String[0]);
+    USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
+    USER_CREATE = User.createUserForTesting(conf, "tbl_create", new String[0]);
+    USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
+    USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
+    USER_QUAL = User.createUserForTesting(conf, "rwpartial", new String[0]);
+    USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    // Create the test table (owner added to the _acl_ table)
+    Admin admin = TEST_UTIL.getHBaseAdmin();
+    HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+    HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
+    hcd.setMaxVersions(100);
+    htd.addFamily(hcd);
+    htd.setOwner(USER_OWNER);
+    admin.createTable(htd, new byte[][] { Bytes.toBytes("s") });
+    TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE.getTableName());
+
+    Region region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE.getTableName()).get(0);
+    RegionCoprocessorHost rcpHost = region.getCoprocessorHost();
+    RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+      Coprocessor.PRIORITY_HIGHEST, 1, TEST_UTIL.getConfiguration());
+
+    // Set up initial grants
+
+    grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(),
+      Permission.Action.ADMIN,
+      Permission.Action.CREATE,
+      Permission.Action.READ,
+      Permission.Action.WRITE);
+
+    grantOnTable(TEST_UTIL, USER_RW.getShortName(),
+      TEST_TABLE.getTableName(), TEST_FAMILY, null,
+      Permission.Action.READ,
+      Permission.Action.WRITE);
+
+    // USER_CREATE is USER_RW plus CREATE permissions
+    grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
+      TEST_TABLE.getTableName(), null, null,
+      Permission.Action.CREATE,
+      Permission.Action.READ,
+      Permission.Action.WRITE);
+
+    grantOnTable(TEST_UTIL, USER_RO.getShortName(),
+      TEST_TABLE.getTableName(), TEST_FAMILY, null,
+      Permission.Action.READ);
+
+    grantOnTable(TEST_UTIL, USER_QUAL.getShortName(),
+      TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+      Permission.Action.READ,
+      Permission.Action.WRITE);
+
+    assertEquals(5, AccessControlLists.getTablePermissions(TEST_UTIL.getConfiguration(),
+      TEST_TABLE.getTableName()).size());
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    // Clean the _acl_ table
+    try {
+      deleteTable(TEST_UTIL, TEST_TABLE.getTableName());
+    } catch (TableNotFoundException ex) {
+      // Test deleted the table, no problem
+      LOG.info("Test deleted table " + TEST_TABLE.getTableName());
+    }
+    // Verify all table/namespace permissions are erased
+    assertEquals(0, AccessControlLists.getTablePermissions(TEST_UTIL.getConfiguration(),
+      TEST_TABLE.getTableName()).size());
+    assertEquals(0, AccessControlLists.getNamespacePermissions(TEST_UTIL.getConfiguration(),
+      TEST_TABLE.getTableName().getNamespaceAsString()).size());
+  }
+
+  @Test
+  public void testCheckPermissions() throws Exception {
+
+    AccessTestAction checkGlobalAdmin = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkGlobalPerms(TEST_UTIL, Permission.Action.ADMIN);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkGlobalAdmin, SUPERUSER, USER_ADMIN);
+    verifyDenied(checkGlobalAdmin, USER_OWNER, USER_CREATE, USER_RW, USER_RO, USER_QUAL,
+      USER_NONE);
+
+    AccessTestAction checkGlobalRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkGlobalPerms(TEST_UTIL, Permission.Action.READ);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkGlobalRead, SUPERUSER, USER_ADMIN);
+    verifyDenied(checkGlobalRead, USER_OWNER, USER_CREATE, USER_RW, USER_RO, USER_QUAL,
+      USER_NONE);
+
+    AccessTestAction checkGlobalReadWrite = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkGlobalPerms(TEST_UTIL, Permission.Action.READ, Permission.Action.WRITE);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkGlobalReadWrite, SUPERUSER, USER_ADMIN);
+    verifyDenied(checkGlobalReadWrite, USER_OWNER, USER_CREATE, USER_RW, USER_RO, USER_QUAL,
+      USER_NONE);
+
+    AccessTestAction checkTableAdmin = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
+          Permission.Action.ADMIN);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkTableAdmin, SUPERUSER, USER_ADMIN, USER_OWNER);
+    verifyDenied(checkTableAdmin, USER_CREATE, USER_RW, USER_RO, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkTableCreate = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
+          Permission.Action.CREATE);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkTableCreate, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE);
+    verifyDenied(checkTableCreate, USER_RW, USER_RO, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkTableRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
+          Permission.Action.READ);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkTableRead, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE);
+    verifyDenied(checkTableRead, USER_RW, USER_RO, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkTableReadWrite = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
+          Permission.Action.READ, Permission.Action.WRITE);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkTableReadWrite, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE);
+    verifyDenied(checkTableReadWrite, USER_RW, USER_RO, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkColumnRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
+          Permission.Action.READ);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkColumnRead, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW,
+      USER_RO);
+    verifyDenied(checkColumnRead, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkColumnReadWrite = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
+          Permission.Action.READ, Permission.Action.WRITE);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkColumnReadWrite, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE,
+      USER_RW);
+    verifyDenied(checkColumnReadWrite, USER_RO, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkQualifierRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+          Permission.Action.READ);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkQualifierRead, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW,
+      USER_RO, USER_QUAL);
+    verifyDenied(checkQualifierRead, USER_NONE);
+
+    AccessTestAction checkQualifierReadWrite = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+          Permission.Action.READ, Permission.Action.WRITE);
+        return null;
+      }
+    };
+
+    verifyAllowed(checkQualifierReadWrite, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE,
+      USER_RW, USER_QUAL);
+    verifyDenied(checkQualifierReadWrite, USER_RO, USER_NONE);
+
+    AccessTestAction checkMultiQualifierRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
+          new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+            Permission.Action.READ),
+          new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
+            Permission.Action.READ), });
+        return null;
+      }
+    };
+
+    verifyAllowed(checkMultiQualifierRead, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE,
+      USER_RW, USER_RO);
+    verifyDenied(checkMultiQualifierRead, USER_QUAL, USER_NONE);
+
+    AccessTestAction checkMultiQualifierReadWrite = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
+            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+              Permission.Action.READ, Permission.Action.WRITE),
+            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
+              Permission.Action.READ, Permission.Action.WRITE), });
+        return null;
+      }
+    };
+
+    verifyAllowed(checkMultiQualifierReadWrite, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE,
+      USER_RW);
+    verifyDenied(checkMultiQualifierReadWrite, USER_RO, USER_QUAL, USER_NONE);
+  }
+
+  /** Test grants and revocations with authorization disabled */
+  @Test
+  public void testPassiveGrantRevoke() throws Exception {
+
+    // Add a test user
+
+    User tblUser = User.createUserForTesting(TEST_UTIL.getConfiguration(), "tbluser",
+      new String[0]);
+
+    // If we check now, the test user won't have permissions
+
+    AccessTestAction checkTableRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
+          Permission.Action.READ);
+        return null;
+      }
+    };
+
+    verifyDenied(tblUser, checkTableRead);
+
+    // An actual read won't be denied
+
+    AccessTestAction tableRead = new AccessTestAction() {
+      @Override
+      public Void run() throws Exception {
+        try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+             Table t = conn.getTable(TEST_TABLE.getTableName())) {
+          t.get(new Get(TEST_ROW).addFamily(TEST_FAMILY));
+        }
+        return null;
+      }
+    };
+
+    verifyAllowed(tblUser, tableRead);
+
+    // Grant read perms to the test user
+
+    grantOnTable(TEST_UTIL, tblUser.getShortName(), TEST_TABLE.getTableName(), TEST_FAMILY,
+      null, Permission.Action.READ);
+
+    // Now both the permission check and actual op will succeed
+
+    verifyAllowed(tblUser, checkTableRead);
+    verifyAllowed(tblUser, tableRead);
+
+    // Revoke read perms from the test user
+
+    revokeFromTable(TEST_UTIL, tblUser.getShortName(), TEST_TABLE.getTableName(), TEST_FAMILY,
+      null, Permission.Action.READ);
+
+    // Now the permission check will indicate revocation but the actual op will still succeed
+
+    verifyDenied(tblUser, checkTableRead);
+    verifyAllowed(tblUser, tableRead);
+  }
+
+  /** Test master observer */
+  @Test
+  public void testPassiveMasterOperations() throws Exception {
+
+    // preCreateTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        ACCESS_CONTROLLER.preCreateTable(ObserverContext.createAndPrepare(CP_ENV, null), htd,
+          null);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preModifyTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+        htd.addFamily(new HColumnDescriptor(TEST_FAMILY2));
+        ACCESS_CONTROLLER.preModifyTable(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName(), htd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDeleteTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName());
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preTruncateTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preTruncateTable(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName());
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preAddColumn
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
+        ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName(), hcd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preModifyColumn
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
+        ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName(), hcd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDeleteColumn
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName(), TEST_FAMILY2);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preEnableTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName());
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDisableTable
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName());
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preMove
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HRegionInfo region = new HRegionInfo(TEST_TABLE.getTableName());
+        ServerName srcServer = ServerName.valueOf("1.1.1.1", 1, 0);
+        ServerName destServer = ServerName.valueOf("2.2.2.2", 2, 0);
+        ACCESS_CONTROLLER.preMove(ObserverContext.createAndPrepare(CP_ENV, null), region,
+          srcServer, destServer);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preAssign
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HRegionInfo region = new HRegionInfo(TEST_TABLE.getTableName());
+        ACCESS_CONTROLLER.preAssign(ObserverContext.createAndPrepare(CP_ENV, null), region);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preUnassign
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HRegionInfo region = new HRegionInfo(TEST_TABLE.getTableName());
+        ACCESS_CONTROLLER.preUnassign(ObserverContext.createAndPrepare(CP_ENV, null), region,
+          true);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preBalance
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBalance(ObserverContext.createAndPrepare(CP_ENV, null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preBalanceSwitch
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBalanceSwitch(ObserverContext.createAndPrepare(CP_ENV, null),
+          true);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preSnapshot
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+          .setName("foo")
+          .build();
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        ACCESS_CONTROLLER.preSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
+          snapshot, htd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preListSnapshot
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+          .setName("foo")
+          .build();
+        ACCESS_CONTROLLER.preListSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
+          snapshot);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preCloneSnapshot
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+          .setName("foo")
+          .build();
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
+          snapshot, htd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preRestoreSnapshot
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+          .setName("foo")
+          .build();
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        ACCESS_CONTROLLER.preRestoreSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
+          snapshot, htd);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDeleteSnapshot
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+          .setName("foo")
+          .build();
+        ACCESS_CONTROLLER.preDeleteSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
+          snapshot);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preGetTableDescriptors
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        List<TableName> tableNamesList = Lists.newArrayList();
+        tableNamesList.add(TEST_TABLE.getTableName());
+        List<HTableDescriptor> descriptors = Lists.newArrayList();
+        ACCESS_CONTROLLER.preGetTableDescriptors(ObserverContext.createAndPrepare(CP_ENV, null),
+          tableNamesList, descriptors, ".+");
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preGetTableNames
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        List<HTableDescriptor> descriptors = Lists.newArrayList();
+        ACCESS_CONTROLLER.preGetTableNames(ObserverContext.createAndPrepare(CP_ENV, null),
+          descriptors, ".+");
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preCreateNamespace
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        NamespaceDescriptor ns = NamespaceDescriptor.create("test").build();
+        ACCESS_CONTROLLER.preCreateNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
+          ns);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDeleteNamespace
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDeleteNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
+          "test");
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preModifyNamespace
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        NamespaceDescriptor ns = NamespaceDescriptor.create("test").build();
+        ACCESS_CONTROLLER.preModifyNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
+          ns);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preGetNamespaceDescriptor
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preGetNamespaceDescriptor(ObserverContext.createAndPrepare(CP_ENV,
+            null),
+          "test");
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preListNamespaceDescriptors
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        List<NamespaceDescriptor> descriptors = Lists.newArrayList();
+        ACCESS_CONTROLLER.preListNamespaceDescriptors(ObserverContext.createAndPrepare(CP_ENV,
+            null),
+          descriptors);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preSetUserQuota
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Quotas quotas = Quotas.newBuilder().build();
+        ACCESS_CONTROLLER.preSetUserQuota(ObserverContext.createAndPrepare(CP_ENV, null),
+          "testuser", quotas);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preSetTableQuota
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Quotas quotas = Quotas.newBuilder().build();
+        ACCESS_CONTROLLER.preSetTableQuota(ObserverContext.createAndPrepare(CP_ENV, null),
+          TEST_TABLE.getTableName(), quotas);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preSetNamespaceQuota
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        Quotas quotas = Quotas.newBuilder().build();
+        ACCESS_CONTROLLER.preSetNamespaceQuota(ObserverContext.createAndPrepare(CP_ENV, null),
+          "test", quotas);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+  }
+
+  /** Test region server observer */
+  @Test
+  public void testPassiveRegionServerOperations() throws Exception {
+    // preStopRegionServer
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preStopRegionServer(ObserverContext.createAndPrepare(RSCP_ENV, null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preMerge
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        HTableDescriptor htd = new HTableDescriptor(TEST_TABLE.getTableName());
+        Region region_a = mock(Region.class);
+        when(region_a.getTableDesc()).thenReturn(htd);
+        Region region_b = mock(Region.class);
+        when(region_b.getTableDesc()).thenReturn(htd);
+        ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null), region_a,
+          region_b);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preRollWALWriterRequest
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preRollWALWriterRequest(ObserverContext.createAndPrepare(RSCP_ENV,
+          null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+  }
+
+  /** Test region observer */
+  @Test
+  public void testPassiveRegionOperations() throws Exception {
+
+    // preOpen
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preOpen(ObserverContext.createAndPrepare(RCP_ENV, null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preFlush
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preFlush(ObserverContext.createAndPrepare(RCP_ENV, null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preSplit
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preGetClosestRowBefore
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preGetClosestRowBefore(ObserverContext.createAndPrepare(RCP_ENV, null),
+          TEST_ROW, TEST_FAMILY, new Result());
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preGetOp
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        List<Cell> cells = Lists.newArrayList();
+        ACCESS_CONTROLLER.preGetOp(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Get(TEST_ROW), cells);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preExists
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preExists(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Get(TEST_ROW), true);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // prePut
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.prePut(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Put(TEST_ROW), new WALEdit(), Durability.USE_DEFAULT);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preDelete
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preDelete(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Delete(TEST_ROW), new WALEdit(), Durability.USE_DEFAULT);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preBatchMutate
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preBatchMutate(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new MiniBatchOperationInProgress<Mutation>(null, null, null, 0, 0));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preCheckAndPut
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preCheckAndPut(ObserverContext.createAndPrepare(RCP_ENV, null),
+          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareFilter.CompareOp.EQUAL,
+          new BinaryComparator("foo".getBytes()), new Put(TEST_ROW), true);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preCheckAndDelete
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preCheckAndDelete(ObserverContext.createAndPrepare(RCP_ENV, null),
+          TEST_ROW, TEST_FAMILY, TEST_Q1, CompareFilter.CompareOp.EQUAL,
+          new BinaryComparator("foo".getBytes()), new Delete(TEST_ROW), true);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preAppend
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preAppend(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Append(TEST_ROW));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preIncrement
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preIncrement(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Increment(TEST_ROW));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preScannerOpen
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preScannerOpen(ObserverContext.createAndPrepare(RCP_ENV, null),
+          new Scan(), mock(RegionScanner.class));
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+    // preBulkLoadHFile
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        List<Pair<byte[], String>> paths = Lists.newArrayList();
+        ACCESS_CONTROLLER.preBulkLoadHFile(ObserverContext.createAndPrepare(RCP_ENV, null),
+          paths);
+        return null;
+      }
+    }, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
+
+  }
+
+  @Test
+  public void testPassiveCellPermissions() throws Exception {
+    final Configuration conf = TEST_UTIL.getConfiguration();
+
+    // store two sets of values, one store with a cell level ACL, and one without
+    verifyAllowed(new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        try(Connection connection = ConnectionFactory.createConnection(conf);
+            Table t = connection.getTable(TEST_TABLE.getTableName())) {
+          Put p;
+          // with ro ACL
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q1, ZERO);
+          p.setACL(USER_NONE.getShortName(), new Permission(Action.READ));
+          t.put(p);
+          // with rw ACL
+          p = new Put(TEST_ROW).add(TEST_FAMILY, TEST_Q2, ZERO);
+          p.setACL(USER_NONE.getShortName(), new Permission(Action.READ, Action.WRITE));
+          t.put(p);
+          // no ACL
+          p = new Put(TEST_ROW)
+            .add(TEST_FAMILY, TEST_Q3, ZERO)
+            .add(TEST_FAMILY, TEST_Q4, ZERO);
+          t.put(p);
+        }
+        return null;
+      }
+    }, USER_OWNER);
+
+    // check that a scan over the test data returns the expected number of KVs
+
+    final List<Cell> scanResults = Lists.newArrayList();
+
+    AccessTestAction scanAction = new AccessTestAction() {
+      @Override
+      public List<Cell> run() throws Exception {
+        Scan scan = new Scan();
+        scan.setStartRow(TEST_ROW);
+        scan.setStopRow(Bytes.add(TEST_ROW, new byte[]{ 0 } ));
+        scan.addFamily(TEST_FAMILY);
+        Connection connection = ConnectionFactory.createConnection(conf);
+        Table t = connection.getTable(TEST_TABLE.getTableName());
+        try {
+          ResultScanner scanner = t.getScanner(scan);
+          Result result = null;
+          do {
+            result = scanner.next();
+            if (result != null) {
+              scanResults.addAll(result.listCells());
+            }
+          } while (result != null);
+        } finally {
+          t.close();
+          connection.close();
+        }
+        return scanResults;
+      }
+    };
+
+    // owner will see all values
+    scanResults.clear();
+    verifyAllowed(scanAction, USER_OWNER);
+    assertEquals(4, scanResults.size());
+
+    // other user will also see 4 values
+    // if cell filtering was active, we would only see 2 values
+    scanResults.clear();
+    verifyAllowed(scanAction, USER_NONE);
+    assertEquals(4, scanResults.size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
new file mode 100644
index 0000000..d5e83de
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestWithDisabledAuthorization.java
@@ -0,0 +1,237 @@
+/**
+ * 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.security.visibility;
+
+import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
+import static org.junit.Assert.*;
+
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+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.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.SecureTestUtil;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.google.protobuf.ByteString;
+
+@Category({SecurityTests.class, LargeTests.class})
+public class TestWithDisabledAuthorization {
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  
+  private static final String CONFIDENTIAL = "confidential";
+  private static final String SECRET = "secret";
+  private static final String PRIVATE = "private";
+  private static final byte[] TEST_FAMILY = Bytes.toBytes("test");
+  private static final byte[] TEST_QUALIFIER = Bytes.toBytes("q");
+  private static final byte[] ZERO = Bytes.toBytes(0L);
+
+
+  @Rule 
+  public final TestName TEST_NAME = new TestName();
+
+  private static User SUPERUSER;
+  private static User USER_RW;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+
+    // Set up superuser
+    SecureTestUtil.configureSuperuser(conf);
+
+    // Install the VisibilityController as a system processor
+    VisibilityTestUtil.enableVisiblityLabels(conf);
+
+    // Now, DISABLE active authorization
+    conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, false);
+
+    TEST_UTIL.startMiniCluster();
+
+    // Wait for the labels table to become available
+    TEST_UTIL.waitUntilAllRegionsAssigned(LABELS_TABLE_NAME);
+
+    // create a set of test users
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
+    USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
+
+    // Define test labels
+    SUPERUSER.runAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        try {
+          VisibilityClient.addLabels(TEST_UTIL.getConfiguration(),
+            new String[] { SECRET, CONFIDENTIAL, PRIVATE });
+          VisibilityClient.setAuths(TEST_UTIL.getConfiguration(),
+            new String[] { SECRET, CONFIDENTIAL },
+            USER_RW.getShortName());
+        } catch (Throwable t) {
+          fail("Should not have failed");          
+        }
+        return null;
+      }
+    });
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testManageUserAuths() throws Throwable {
+    // Even though authorization is disabled, we should be able to manage user auths
+
+    SUPERUSER.runAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        try {
+          VisibilityClient.setAuths(TEST_UTIL.getConfiguration(),
+            new String[] { SECRET, CONFIDENTIAL },
+            USER_RW.getShortName());
+        } catch (Throwable t) {
+          fail("Should not have failed");          
+        }
+        return null;
+      }
+    });
+
+    PrivilegedExceptionAction<List<String>> getAuths =
+      new PrivilegedExceptionAction<List<String>>() {
+        public List<String> run() throws Exception {
+          GetAuthsResponse authsResponse = null;
+          try {
+            authsResponse = VisibilityClient.getAuths(TEST_UTIL.getConfiguration(),
+              USER_RW.getShortName());
+          } catch (Throwable t) {
+            fail("Should not have failed");
+          }
+          List<String> authsList = new ArrayList<String>();
+          for (ByteString authBS : authsResponse.getAuthList()) {
+            authsList.add(Bytes.toString(authBS.toByteArray()));
+          }
+          return authsList;
+        }
+      };
+
+    List<String> authsList = SUPERUSER.runAs(getAuths);
+    assertEquals(2, authsList.size());
+    assertTrue(authsList.contains(SECRET));
+    assertTrue(authsList.contains(CONFIDENTIAL));
+
+    SUPERUSER.runAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        try {
+          VisibilityClient.clearAuths(TEST_UTIL.getConfiguration(),
+            new String[] { SECRET },
+            USER_RW.getShortName());
+        } catch (Throwable t) {
+          fail("Should not have failed");          
+        }
+        return null;
+      }
+    });
+
+    authsList = SUPERUSER.runAs(getAuths);
+    assertEquals(1, authsList.size());
+    assertTrue(authsList.contains(CONFIDENTIAL));
+
+    SUPERUSER.runAs(new PrivilegedExceptionAction<Void>() {
+      public Void run() throws Exception {
+        try {
+          VisibilityClient.clearAuths(TEST_UTIL.getConfiguration(),
+            new String[] { CONFIDENTIAL },
+            USER_RW.getShortName());
+        } catch (Throwable t) {
+          fail("Should not have failed");          
+        }
+        return null;
+      }
+    });
+
+    authsList = SUPERUSER.runAs(getAuths);
+    assertEquals(0, authsList.size());
+  }
+
+  @Test
+  public void testPassiveVisibility() throws Exception {
+    // No values should be filtered regardless of authorization if we are passive
+    try (Table t = createTableAndWriteDataWithLabels(
+      TableName.valueOf(TEST_NAME.getMethodName()),
+        SECRET,
+        PRIVATE,
+        SECRET + "|" + CONFIDENTIAL,
+        PRIVATE + "|" + CONFIDENTIAL)) {
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations());
+      try (ResultScanner scanner = t.getScanner(s)) {
+        Result[] next = scanner.next(10);
+        assertEquals(next.length, 4);
+      }
+      s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET));
+      try (ResultScanner scanner = t.getScanner(s)) {
+        Result[] next = scanner.next(10);
+        assertEquals(next.length, 4);
+      }
+      s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
+      try (ResultScanner scanner = t.getScanner(s)) {
+        Result[] next = scanner.next(10);
+        assertEquals(next.length, 4);
+      }
+      s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL, PRIVATE));
+      try (ResultScanner scanner = t.getScanner(s)) {
+        Result[] next = scanner.next(10);
+        assertEquals(next.length, 4);
+      }
+    }
+  }
+
+  static Table createTableAndWriteDataWithLabels(TableName tableName, String... labelExps)
+      throws Exception {
+    List<Put> puts = new ArrayList<Put>();
+    for (int i = 0; i < labelExps.length; i++) {
+      Put put = new Put(Bytes.toBytes("row" + (i+1)));
+      put.addColumn(TEST_FAMILY, TEST_QUALIFIER, HConstants.LATEST_TIMESTAMP, ZERO);
+      put.setCellVisibility(new CellVisibility(labelExps[i]));
+      puts.add(put);
+    }
+    Table table = TEST_UTIL.createTable(tableName, TEST_FAMILY);
+    table.put(puts);
+    return table;
+  }
+}


[12/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
new file mode 100644
index 0000000..13f7bfa
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -0,0 +1,721 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.IOException;
+import java.io.FileNotFoundException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+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.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
+import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+
+/**
+ * WAL implementation of the ProcedureStore.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class WALProcedureStore implements ProcedureStore {
+  private static final Log LOG = LogFactory.getLog(WALProcedureStore.class);
+
+  public interface LeaseRecovery {
+    void recoverFileLease(FileSystem fs, Path path) throws IOException;
+  }
+
+  private static final int MAX_RETRIES_BEFORE_ABORT = 3;
+
+  private static final String SYNC_WAIT_MSEC_CONF_KEY = "hbase.procedure.store.wal.sync.wait.msec";
+  private static final int DEFAULT_SYNC_WAIT_MSEC = 100;
+
+  private final CopyOnWriteArrayList<ProcedureStoreListener> listeners =
+    new CopyOnWriteArrayList<ProcedureStoreListener>();
+
+  private final LinkedList<ProcedureWALFile> logs = new LinkedList<ProcedureWALFile>();
+  private final ProcedureStoreTracker storeTracker = new ProcedureStoreTracker();
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final ReentrantLock lock = new ReentrantLock();
+  private final Condition waitCond = lock.newCondition();
+  private final Condition slotCond = lock.newCondition();
+  private final Condition syncCond = lock.newCondition();
+
+  private final LeaseRecovery leaseRecovery;
+  private final Configuration conf;
+  private final FileSystem fs;
+  private final Path logDir;
+
+  private AtomicBoolean inSync = new AtomicBoolean(false);
+  private ArrayBlockingQueue<ByteSlot> slotsCache = null;
+  private Set<ProcedureWALFile> corruptedLogs = null;
+  private FSDataOutputStream stream = null;
+  private long totalSynced = 0;
+  private long flushLogId = 0;
+  private int slotIndex = 0;
+  private Thread syncThread;
+  private ByteSlot[] slots;
+  private int syncWaitMsec;
+
+  public WALProcedureStore(final Configuration conf, final FileSystem fs, final Path logDir,
+      final LeaseRecovery leaseRecovery) {
+    this.fs = fs;
+    this.conf = conf;
+    this.logDir = logDir;
+    this.leaseRecovery = leaseRecovery;
+  }
+
+  @Override
+  public void start(int numSlots) throws IOException {
+    if (running.getAndSet(true)) {
+      return;
+    }
+
+    // Init buffer slots
+    slots = new ByteSlot[numSlots];
+    slotsCache = new ArrayBlockingQueue(numSlots, true);
+    while (slotsCache.remainingCapacity() > 0) {
+      slotsCache.offer(new ByteSlot());
+    }
+
+    // Tunings
+    syncWaitMsec = conf.getInt(SYNC_WAIT_MSEC_CONF_KEY, DEFAULT_SYNC_WAIT_MSEC);
+
+    // Init sync thread
+    syncThread = new Thread("WALProcedureStoreSyncThread") {
+      @Override
+      public void run() {
+        while (running.get()) {
+          try {
+            syncLoop();
+          } catch (IOException e) {
+            LOG.error("got an exception from the sync-loop", e);
+            sendAbortProcessSignal();
+          }
+        }
+      }
+    };
+    syncThread.start();
+  }
+
+  @Override
+  public void stop(boolean abort) {
+    if (!running.getAndSet(false)) {
+      return;
+    }
+
+    LOG.info("Stopping the WAL Procedure Store");
+    if (lock.tryLock()) {
+      try {
+        waitCond.signalAll();
+      } finally {
+        lock.unlock();
+      }
+    }
+
+    if (!abort) {
+      try {
+        syncThread.join();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      }
+    }
+
+    // Close the writer
+    closeStream();
+
+    // Close the old logs
+    // they should be already closed, this is just in case the load fails
+    // and we call start() and then stop()
+    for (ProcedureWALFile log: logs) {
+      log.close();
+    }
+    logs.clear();
+  }
+
+  @Override
+  public boolean isRunning() {
+    return running.get();
+  }
+
+  @Override
+  public int getNumThreads() {
+    return slots == null ? 0 : slots.length;
+  }
+
+  public ProcedureStoreTracker getStoreTracker() {
+    return storeTracker;
+  }
+
+  @Override
+  public void registerListener(ProcedureStoreListener listener) {
+    this.listeners.add(listener);
+  }
+
+  @Override
+  public boolean unregisterListener(ProcedureStoreListener listener) {
+    return this.listeners.remove(listener);
+  }
+
+  @Override
+  public void recoverLease() throws IOException {
+    LOG.info("Starting WAL Procedure Store lease recovery");
+    FileStatus[] oldLogs = getLogFiles();
+    while (running.get()) {
+      // Get Log-MaxID and recover lease on old logs
+      flushLogId = initOldLogs(oldLogs) + 1;
+
+      // Create new state-log
+      if (!rollWriter(flushLogId)) {
+        // someone else has already created this log
+        LOG.debug("someone else has already created log " + flushLogId);
+        continue;
+      }
+
+      // We have the lease on the log
+      oldLogs = getLogFiles();
+      if (getMaxLogId(oldLogs) > flushLogId) {
+        // Someone else created new logs
+        LOG.debug("someone else created new logs. expected maxLogId < " + flushLogId);
+        logs.getLast().removeFile();
+        continue;
+      }
+
+      LOG.info("lease acquired flushLogId=" + flushLogId);
+      break;
+    }
+  }
+
+  @Override
+  public Iterator<Procedure> load() throws IOException {
+    if (logs.isEmpty()) {
+      throw new RuntimeException("recoverLease() must be called before loading data");
+    }
+
+    // Nothing to do, If we have only the current log.
+    if (logs.size() == 1) {
+      LOG.debug("No state logs to replay");
+      return null;
+    }
+
+    // Load the old logs
+    final ArrayList<ProcedureWALFile> toRemove = new ArrayList<ProcedureWALFile>();
+    Iterator<ProcedureWALFile> it = logs.descendingIterator();
+    it.next(); // Skip the current log
+    try {
+      return ProcedureWALFormat.load(it, storeTracker, new ProcedureWALFormat.Loader() {
+        @Override
+        public void removeLog(ProcedureWALFile log) {
+          toRemove.add(log);
+        }
+
+        @Override
+        public void markCorruptedWAL(ProcedureWALFile log, IOException e) {
+          if (corruptedLogs == null) {
+            corruptedLogs = new HashSet<ProcedureWALFile>();
+          }
+          corruptedLogs.add(log);
+          // TODO: sideline corrupted log
+        }
+      });
+    } finally {
+      if (!toRemove.isEmpty()) {
+        for (ProcedureWALFile log: toRemove) {
+          removeLogFile(log);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void insert(final Procedure proc, final Procedure[] subprocs) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("insert " + proc + " subproc=" + Arrays.toString(subprocs));
+    }
+
+    ByteSlot slot = acquireSlot();
+    long logId = -1;
+    try {
+      // Serialize the insert
+      if (subprocs != null) {
+        ProcedureWALFormat.writeInsert(slot, proc, subprocs);
+      } else {
+        assert !proc.hasParent();
+        ProcedureWALFormat.writeInsert(slot, proc);
+      }
+
+      // Push the transaction data and wait until it is persisted
+      logId = pushData(slot);
+    } catch (IOException e) {
+      // We are not able to serialize the procedure.
+      // this is a code error, and we are not able to go on.
+      LOG.fatal("Unable to serialize one of the procedure: proc=" + proc +
+                " subprocs=" + Arrays.toString(subprocs), e);
+      throw new RuntimeException(e);
+    } finally {
+      releaseSlot(slot);
+    }
+
+    // Update the store tracker
+    synchronized (storeTracker) {
+      if (logId == flushLogId) {
+        storeTracker.insert(proc, subprocs);
+      }
+    }
+  }
+
+  @Override
+  public void update(final Procedure proc) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("update " + proc);
+    }
+
+    ByteSlot slot = acquireSlot();
+    long logId = -1;
+    try {
+      // Serialize the update
+      ProcedureWALFormat.writeUpdate(slot, proc);
+
+      // Push the transaction data and wait until it is persisted
+      logId = pushData(slot);
+    } catch (IOException e) {
+      // We are not able to serialize the procedure.
+      // this is a code error, and we are not able to go on.
+      LOG.fatal("Unable to serialize the procedure: " + proc, e);
+      throw new RuntimeException(e);
+    } finally {
+      releaseSlot(slot);
+    }
+
+    // Update the store tracker
+    boolean removeOldLogs = false;
+    synchronized (storeTracker) {
+      if (logId == flushLogId) {
+        storeTracker.update(proc);
+        removeOldLogs = storeTracker.isUpdated();
+      }
+    }
+
+    if (removeOldLogs) {
+      removeAllLogs(logId - 1);
+    }
+  }
+
+  @Override
+  public void delete(final long procId) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("delete " + procId);
+    }
+
+    ByteSlot slot = acquireSlot();
+    long logId = -1;
+    try {
+      // Serialize the delete
+      ProcedureWALFormat.writeDelete(slot, procId);
+
+      // Push the transaction data and wait until it is persisted
+      logId = pushData(slot);
+    } catch (IOException e) {
+      // We are not able to serialize the procedure.
+      // this is a code error, and we are not able to go on.
+      LOG.fatal("Unable to serialize the procedure: " + procId, e);
+      throw new RuntimeException(e);
+    } finally {
+      releaseSlot(slot);
+    }
+
+    boolean removeOldLogs = false;
+    synchronized (storeTracker) {
+      if (logId == flushLogId) {
+        storeTracker.delete(procId);
+        if (storeTracker.isEmpty()) {
+          removeOldLogs = rollWriterOrDie(logId + 1);
+        }
+      }
+    }
+
+    if (removeOldLogs) {
+      removeAllLogs(logId);
+    }
+  }
+
+  private ByteSlot acquireSlot() {
+    ByteSlot slot = slotsCache.poll();
+    return slot != null ? slot : new ByteSlot();
+  }
+
+  private void releaseSlot(final ByteSlot slot) {
+    slot.reset();
+    slotsCache.offer(slot);
+  }
+
+  private long pushData(final ByteSlot slot) {
+    assert !logs.isEmpty() : "recoverLease() must be called before inserting data";
+    long logId = -1;
+
+    lock.lock();
+    try {
+      // Wait for the sync to be completed
+      while (true) {
+        if (inSync.get()) {
+          syncCond.await();
+        } else if (slotIndex == slots.length) {
+          slotCond.signal();
+          syncCond.await();
+        } else {
+          break;
+        }
+      }
+
+      slots[slotIndex++] = slot;
+      logId = flushLogId;
+
+      // Notify that there is new data
+      if (slotIndex == 1) {
+        waitCond.signal();
+      }
+
+      // Notify that the slots are full
+      if (slotIndex == slots.length) {
+        slotCond.signal();
+      }
+      syncCond.await();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      sendAbortProcessSignal();
+    } finally {
+      lock.unlock();
+    }
+    return logId;
+  }
+
+  private void syncLoop() throws IOException {
+    inSync.set(false);
+    while (running.get()) {
+      lock.lock();
+      try {
+        // Wait until new data is available
+        if (slotIndex == 0) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Waiting for data. flushed=" + StringUtils.humanSize(totalSynced));
+          }
+          waitCond.await();
+          if (slotIndex == 0) {
+            // no data.. probably a stop()
+            continue;
+          }
+        }
+
+        // Wait SYNC_WAIT_MSEC or the signal of "slots full" before flushing
+        slotCond.await(syncWaitMsec, TimeUnit.MILLISECONDS);
+
+        inSync.set(true);
+        totalSynced += syncSlots();
+        slotIndex = 0;
+        inSync.set(false);
+        syncCond.signalAll();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        sendAbortProcessSignal();
+      } finally {
+        lock.unlock();
+      }
+    }
+  }
+
+  private long syncSlots() {
+    int retry = 0;
+    long totalSynced = 0;
+    do {
+      try {
+        totalSynced = syncSlots(stream, slots, 0, slotIndex);
+        break;
+      } catch (Throwable e) {
+        if (++retry == MAX_RETRIES_BEFORE_ABORT) {
+          LOG.error("sync slot failed, abort.", e);
+          sendAbortProcessSignal();
+        }
+      }
+    } while (running.get());
+    return totalSynced;
+  }
+
+  protected long syncSlots(FSDataOutputStream stream, ByteSlot[] slots, int offset, int count)
+      throws IOException {
+    long totalSynced = 0;
+    for (int i = 0; i < count; ++i) {
+      ByteSlot data = slots[offset + i];
+      data.writeTo(stream);
+      totalSynced += data.size();
+    }
+    stream.hsync();
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Sync slots=" + count + '/' + slots.length +
+                " flushed=" + StringUtils.humanSize(totalSynced));
+    }
+    return totalSynced;
+  }
+
+  private void sendAbortProcessSignal() {
+    if (!this.listeners.isEmpty()) {
+      for (ProcedureStoreListener listener : this.listeners) {
+        listener.abortProcess();
+      }
+    }
+  }
+
+  private boolean rollWriterOrDie(final long logId) {
+    try {
+      return rollWriter(logId);
+    } catch (IOException e) {
+      LOG.warn("Unable to roll the log", e);
+      sendAbortProcessSignal();
+      return false;
+    }
+  }
+
+  private boolean rollWriter(final long logId) throws IOException {
+    ProcedureWALHeader header = ProcedureWALHeader.newBuilder()
+      .setVersion(ProcedureWALFormat.HEADER_VERSION)
+      .setType(ProcedureWALFormat.LOG_TYPE_STREAM)
+      .setMinProcId(storeTracker.getMinProcId())
+      .setLogId(logId)
+      .build();
+
+    FSDataOutputStream newStream = null;
+    Path newLogFile = null;
+    long startPos = -1;
+    try {
+      newLogFile = getLogFilePath(logId);
+      newStream = fs.create(newLogFile, false);
+      ProcedureWALFormat.writeHeader(newStream, header);
+      startPos = newStream.getPos();
+    } catch (FileAlreadyExistsException e) {
+      LOG.error("Log file with id=" + logId + " already exists", e);
+      return false;
+    }
+    lock.lock();
+    try {
+      closeStream();
+      synchronized (storeTracker) {
+        storeTracker.resetUpdates();
+      }
+      stream = newStream;
+      flushLogId = logId;
+      totalSynced = 0;
+      logs.add(new ProcedureWALFile(fs, newLogFile, header, startPos));
+    } finally {
+      lock.unlock();
+    }
+    LOG.info("Roll new state log: " + logId);
+    return true;
+  }
+
+  private void closeStream() {
+    try {
+      if (stream != null) {
+        try {
+          ProcedureWALFormat.writeTrailer(stream, storeTracker);
+        } catch (IOException e) {
+          LOG.warn("Unable to write the trailer: " + e.getMessage());
+        }
+        stream.close();
+      }
+    } catch (IOException e) {
+      LOG.error("Unable to close the stream", e);
+    } finally {
+      stream = null;
+    }
+  }
+
+  private void removeAllLogs(long lastLogId) {
+    LOG.info("Remove all state logs with ID less then " + lastLogId);
+    while (!logs.isEmpty()) {
+      ProcedureWALFile log = logs.getFirst();
+      if (lastLogId < log.getLogId()) {
+        break;
+      }
+
+      removeLogFile(log);
+    }
+  }
+
+  private boolean removeLogFile(final ProcedureWALFile log) {
+    try {
+      LOG.debug("remove log: " + log);
+      log.removeFile();
+      logs.remove(log);
+    } catch (IOException e) {
+      LOG.error("unable to remove log " + log, e);
+      return false;
+    }
+    return true;
+  }
+
+  public Set<ProcedureWALFile> getCorruptedLogs() {
+    return corruptedLogs;
+  }
+
+  // ==========================================================================
+  //  FileSystem Log Files helpers
+  // ==========================================================================
+  public Path getLogDir() {
+    return this.logDir;
+  }
+
+  public FileSystem getFileSystem() {
+    return this.fs;
+  }
+
+  protected Path getLogFilePath(final long logId) throws IOException {
+    return new Path(logDir, String.format("state-%020d.log", logId));
+  }
+
+  private static long getLogIdFromName(final String name) {
+    int end = name.lastIndexOf(".log");
+    int start = name.lastIndexOf('-') + 1;
+    while (start < end) {
+      if (name.charAt(start) != '0')
+        break;
+      start++;
+    }
+    return Long.parseLong(name.substring(start, end));
+  }
+
+  private FileStatus[] getLogFiles() throws IOException {
+    try {
+      return fs.listStatus(logDir, new PathFilter() {
+        @Override
+        public boolean accept(Path path) {
+          String name = path.getName();
+          return name.startsWith("state-") && name.endsWith(".log");
+        }
+      });
+    } catch (FileNotFoundException e) {
+      LOG.warn("log directory not found: " + e.getMessage());
+      return null;
+    }
+  }
+
+  private long getMaxLogId(final FileStatus[] logFiles) {
+    long maxLogId = 0;
+    if (logFiles != null && logFiles.length > 0) {
+      for (int i = 0; i < logFiles.length; ++i) {
+        maxLogId = Math.max(maxLogId, getLogIdFromName(logFiles[i].getPath().getName()));
+      }
+    }
+    return maxLogId;
+  }
+
+  /**
+   * @return Max-LogID of the specified log file set
+   */
+  private long initOldLogs(final FileStatus[] logFiles) throws IOException {
+    this.logs.clear();
+
+    long maxLogId = 0;
+    if (logFiles != null && logFiles.length > 0) {
+      for (int i = 0; i < logFiles.length; ++i) {
+        final Path logPath = logFiles[i].getPath();
+        leaseRecovery.recoverFileLease(fs, logPath);
+        maxLogId = Math.max(maxLogId, getLogIdFromName(logPath.getName()));
+
+        ProcedureWALFile log = initOldLog(logFiles[i]);
+        if (log != null) {
+          this.logs.add(log);
+        }
+      }
+      Collections.sort(this.logs);
+      initTrackerFromOldLogs();
+    }
+    return maxLogId;
+  }
+
+  private void initTrackerFromOldLogs() {
+    // TODO: Load the most recent tracker available
+    if (!logs.isEmpty()) {
+      ProcedureWALFile log = logs.getLast();
+      try {
+        log.readTracker(storeTracker);
+      } catch (IOException e) {
+        LOG.error("Unable to read tracker for " + log, e);
+        // try the next one...
+        storeTracker.clear();
+        storeTracker.setPartialFlag(true);
+      }
+    }
+  }
+
+  private ProcedureWALFile initOldLog(final FileStatus logFile) throws IOException {
+    ProcedureWALFile log = new ProcedureWALFile(fs, logFile);
+    if (logFile.getLen() == 0) {
+      LOG.warn("Remove uninitialized log " + logFile);
+      log.removeFile();
+      return null;
+    }
+
+    LOG.debug("opening state-log: " + logFile);
+    try {
+      log.open();
+    } catch (ProcedureWALFormat.InvalidWALDataException e) {
+      LOG.warn("Remove uninitialized log " + logFile, e);
+      log.removeFile();
+      return null;
+    } catch (IOException e) {
+      String msg = "Unable to read state log: " + logFile;
+      LOG.error(msg, e);
+      throw new IOException(msg, e);
+    }
+
+    if (log.isCompacted()) {
+      try {
+        log.readTrailer();
+      } catch (IOException e) {
+        // unfinished compacted log throw it away
+        LOG.warn("Unfinished compacted log " + logFile, e);
+        log.removeFile();
+        return null;
+      }
+    }
+    return log;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/ByteSlot.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/ByteSlot.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/ByteSlot.java
new file mode 100644
index 0000000..8904116
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/ByteSlot.java
@@ -0,0 +1,111 @@
+/**
+ * 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.procedure2.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Similar to the ByteArrayOutputStream, with the exception that we can prepend an header.
+ * e.g. you write some data and you want to prepend an header that contains the data len or cksum.
+ * <code>
+ * ByteSlot slot = new ByteSlot();
+ * // write data
+ * slot.write(...);
+ * slot.write(...);
+ * // write header with the size of the written data
+ * slot.markHead();
+ * slot.write(Bytes.toBytes(slot.size()));
+ * // flush to stream as [header, data]
+ * slot.writeTo(stream);
+ * </code>
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ByteSlot extends OutputStream {
+  private static final int DOUBLE_GROW_LIMIT = 1 << 20;
+  private static final int GROW_ALIGN = 128;
+
+  private byte[] buf;
+  private int head;
+  private int size;
+
+  public void reset() {
+    head = 0;
+    size = 0;
+  }
+
+  public void markHead() {
+    head = size;
+  }
+
+  public int getHead() {
+    return head;
+  }
+
+  public int size() {
+    return size;
+  }
+
+  public byte[] getBuffer() {
+    return buf;
+  }
+
+  public void writeAt(int offset, int b) {
+    head = Math.min(head, offset);
+    buf[offset] = (byte)b;
+  }
+
+  public void write(int b) {
+    ensureCapacity(size + 1);
+    buf[size++] = (byte)b;
+  }
+
+  public void write(byte[] b, int off, int len) {
+    ensureCapacity(size + len);
+    System.arraycopy(b, off, buf, size, len);
+    size += len;
+  }
+
+  public void writeTo(final OutputStream stream) throws IOException {
+    if (head != 0) {
+      stream.write(buf, head, size - head);
+      stream.write(buf, 0, head);
+    } else {
+      stream.write(buf, 0, size);
+    }
+  }
+
+  private void ensureCapacity(int minCapacity) {
+    minCapacity = (minCapacity + (GROW_ALIGN - 1)) & -GROW_ALIGN;
+    if (buf == null) {
+      buf = new byte[minCapacity];
+    } else if (minCapacity > buf.length) {
+      int newCapacity = buf.length << 1;
+      if (minCapacity > newCapacity || newCapacity > DOUBLE_GROW_LIMIT) {
+        newCapacity = minCapacity;
+      }
+      buf = Arrays.copyOf(buf, newCapacity);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/StringUtils.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/StringUtils.java
new file mode 100644
index 0000000..97134c2
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/StringUtils.java
@@ -0,0 +1,80 @@
+/**
+ * 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.procedure2.util;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class StringUtils {
+  private StringUtils() {}
+
+  public static String humanTimeDiff(long timeDiff) {
+    StringBuilder buf = new StringBuilder();
+    long hours = timeDiff / (60*60*1000);
+    long rem = (timeDiff % (60*60*1000));
+    long minutes =  rem / (60*1000);
+    rem = rem % (60*1000);
+    float seconds = rem / 1000.0f;
+
+    if (hours != 0){
+      buf.append(hours);
+      buf.append("hrs, ");
+    }
+    if (minutes != 0){
+      buf.append(minutes);
+      buf.append("mins, ");
+    }
+    if (hours > 0 || minutes > 0) {
+      buf.append(seconds);
+      buf.append("sec");
+    } else {
+      buf.append(String.format("%.4fsec", seconds));
+    }
+    return buf.toString();
+  }
+
+  public static String humanSize(double size) {
+    if (size >= (1L << 40)) return String.format("%.1fT", size / (1L << 40));
+    if (size >= (1L << 30)) return String.format("%.1fG", size / (1L << 30));
+    if (size >= (1L << 20)) return String.format("%.1fM", size / (1L << 20));
+    if (size >= (1L << 10)) return String.format("%.1fK", size / (1L << 10));
+    return String.format("%.0f", size);
+  }
+
+  public static boolean isEmpty(final String input) {
+    return input == null || input.length() == 0;
+  }
+
+  public static String buildString(final String... parts) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < parts.length; ++i) {
+      sb.append(parts[i]);
+    }
+    return sb.toString();
+  }
+
+  public static StringBuilder appendStrings(final StringBuilder sb, final String... parts) {
+    for (int i = 0; i < parts.length; ++i) {
+      sb.append(parts[i]);
+    }
+    return sb;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/TimeoutBlockingQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/TimeoutBlockingQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/TimeoutBlockingQueue.java
new file mode 100644
index 0000000..f710ef4
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/util/TimeoutBlockingQueue.java
@@ -0,0 +1,217 @@
+/**
+ * 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.procedure2.util;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class TimeoutBlockingQueue<E> {
+  public static interface TimeoutRetriever<T> {
+    long getTimeout(T object);
+    TimeUnit getTimeUnit(T object);
+  }
+
+  private final ReentrantLock lock = new ReentrantLock();
+  private final Condition waitCond = lock.newCondition();
+  private final TimeoutRetriever<? super E> timeoutRetriever;
+
+  private E[] objects;
+  private int head = 0;
+  private int tail = 0;
+
+  public TimeoutBlockingQueue(TimeoutRetriever<? super E> timeoutRetriever) {
+    this(32, timeoutRetriever);
+  }
+
+  @SuppressWarnings("unchecked")
+  public TimeoutBlockingQueue(int capacity, TimeoutRetriever<? super E> timeoutRetriever) {
+    this.objects = (E[])new Object[capacity];
+    this.timeoutRetriever = timeoutRetriever;
+  }
+
+  public void dump() {
+    for (int i = 0; i < objects.length; ++i) {
+      if (i == head) {
+        System.out.print("[" + objects[i] + "] ");
+      } else if (i == tail) {
+        System.out.print("]" + objects[i] + "[ ");
+      } else {
+        System.out.print(objects[i] + " ");
+      }
+    }
+    System.out.println();
+  }
+
+  public void clear() {
+    lock.lock();
+    try {
+      if (head != tail) {
+        for (int i = head; i < tail; ++i) {
+          objects[i] = null;
+        }
+        head = 0;
+        tail = 0;
+        waitCond.signal();
+      }
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  public void add(E e) {
+    if (e == null) throw new NullPointerException();
+
+    lock.lock();
+    try {
+      addElement(e);
+      waitCond.signal();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
+  public E poll() {
+    lock.lock();
+    try {
+      if (isEmpty()) {
+        waitCond.await();
+        return null;
+      }
+
+      E elem = objects[head];
+      long nanos = getNanosTimeout(elem);
+      nanos = waitCond.awaitNanos(nanos);
+      return nanos > 0 ? null : removeFirst();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      return null;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  public int size() {
+    return tail - head;
+  }
+
+  public boolean isEmpty() {
+    return (tail - head) == 0;
+  }
+
+  public void signalAll() {
+    lock.lock();
+    try {
+      waitCond.signalAll();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  private void addElement(E elem) {
+    int size = (tail - head);
+    if ((objects.length - size) == 0) {
+      int capacity = size + ((size < 64) ? (size + 2) : (size >> 1));
+      E[] newObjects = (E[])new Object[capacity];
+
+      if (compareTimeouts(objects[tail - 1], elem) <= 0) {
+        // Append
+        System.arraycopy(objects, head, newObjects, 0, tail);
+        tail -= head;
+        newObjects[tail++] = elem;
+      } else if (compareTimeouts(objects[head], elem) > 0) {
+        // Prepend
+        System.arraycopy(objects, head, newObjects, 1, tail);
+        newObjects[0] = elem;
+        tail -= (head - 1);
+      } else {
+        // Insert in the middle
+        int index = upperBound(head, tail - 1, elem);
+        int newIndex = (index - head);
+        System.arraycopy(objects, head, newObjects, 0, newIndex);
+        newObjects[newIndex] = elem;
+        System.arraycopy(objects, index, newObjects, newIndex + 1, tail - index);
+        tail -= (head - 1);
+      }
+      head = 0;
+      objects = newObjects;
+    } else {
+      if (tail == objects.length) {
+        // shift down |-----AAAAAAA|
+        tail -= head;
+        System.arraycopy(objects, head, objects, 0, tail);
+        head = 0;
+      }
+
+      if (tail == head || compareTimeouts(objects[tail - 1], elem) <= 0) {
+        // Append
+        objects[tail++] = elem;
+      } else if (head > 0 && compareTimeouts(objects[head], elem) > 0) {
+        // Prepend
+        objects[--head] = elem;
+      } else {
+        // Insert in the middle
+        int index = upperBound(head, tail - 1, elem);
+        System.arraycopy(objects, index, objects, index + 1, tail - index);
+        objects[index] = elem;
+        tail++;
+      }
+    }
+  }
+
+  private E removeFirst() {
+    E elem = objects[head];
+    objects[head] = null;
+    head = (head + 1) % objects.length;
+    if (head == 0) tail = 0;
+    return elem;
+  }
+
+  private int upperBound(int start, int end, E key) {
+    while (start < end) {
+      int mid = (start + end) >>> 1;
+      E mitem = objects[mid];
+      int cmp = compareTimeouts(mitem, key);
+      if (cmp > 0) {
+        end = mid;
+      } else {
+        start = mid + 1;
+      }
+    }
+    return start;
+  }
+
+  private int compareTimeouts(final E a, final E b) {
+    long t1 = getNanosTimeout(a);
+    long t2 = getNanosTimeout(b);
+    return (t1 < t2) ? -1 : (t1 > t2) ? 1 : 0;
+  }
+
+  private long getNanosTimeout(final E obj) {
+    TimeUnit unit = timeoutRetriever.getTimeUnit(obj);
+    long timeout = timeoutRetriever.getTimeout(obj);
+    return unit.toNanos(timeout);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
new file mode 100644
index 0000000..6e7306c
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -0,0 +1,163 @@
+/**
+ * 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.procedure2;
+
+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.util.Threads;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+
+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 ProcedureTestingUtility {
+  private static final Log LOG = LogFactory.getLog(ProcedureTestingUtility.class);
+
+  private ProcedureTestingUtility() {
+  }
+
+  public static ProcedureStore createStore(final Configuration conf, final FileSystem fs,
+      final Path baseDir) throws IOException {
+    return createWalStore(conf, fs, baseDir);
+  }
+
+  public static WALProcedureStore createWalStore(final Configuration conf, final FileSystem fs,
+      final Path logDir) throws IOException {
+    return new WALProcedureStore(conf, fs, logDir, new WALProcedureStore.LeaseRecovery() {
+      @Override
+      public void recoverFileLease(FileSystem fs, Path path) throws IOException {
+        // no-op
+      }
+    });
+  }
+
+  public static <TEnv> void restart(ProcedureExecutor<TEnv> procExecutor)
+      throws Exception {
+    restart(procExecutor, null);
+  }
+
+  public static <TEnv> void restart(ProcedureExecutor<TEnv> procExecutor,
+      Runnable beforeStartAction) throws Exception {
+    ProcedureStore procStore = procExecutor.getStore();
+    int storeThreads = procExecutor.getNumThreads();
+    int execThreads = procExecutor.getNumThreads();
+    // stop
+    procExecutor.stop();
+    procStore.stop(false);
+    procExecutor.join();
+    // nothing running...
+    if (beforeStartAction != null) {
+      beforeStartAction.run();
+    }
+    // re-start
+    procStore.start(storeThreads);
+    procExecutor.start(execThreads);
+  }
+
+  public static <TEnv> void setKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
+      boolean value) {
+    if (procExecutor.testing == null) {
+      procExecutor.testing = new ProcedureExecutor.Testing();
+    }
+    procExecutor.testing.killBeforeStoreUpdate = value;
+    LOG.warn("Set Kill before store update to: " + procExecutor.testing.killBeforeStoreUpdate);
+  }
+
+  public static <TEnv> void setToggleKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
+      boolean value) {
+    if (procExecutor.testing == null) {
+      procExecutor.testing = new ProcedureExecutor.Testing();
+    }
+    procExecutor.testing.toggleKillBeforeStoreUpdate = value;
+  }
+
+  public static <TEnv> void toggleKillBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor) {
+    if (procExecutor.testing == null) {
+      procExecutor.testing = new ProcedureExecutor.Testing();
+    }
+    procExecutor.testing.killBeforeStoreUpdate = !procExecutor.testing.killBeforeStoreUpdate;
+    LOG.warn("Set Kill before store update to: " + procExecutor.testing.killBeforeStoreUpdate);
+  }
+
+  public static <TEnv> void setKillAndToggleBeforeStoreUpdate(ProcedureExecutor<TEnv> procExecutor,
+      boolean value) {
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, value);
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, value);
+  }
+
+  public static <TEnv> long submitAndWait(ProcedureExecutor<TEnv> procExecutor, Procedure proc) {
+    long procId = procExecutor.submitProcedure(proc);
+    waitProcedure(procExecutor, procId);
+    return procId;
+  }
+
+  public static <TEnv> void waitProcedure(ProcedureExecutor<TEnv> procExecutor, long procId) {
+    while (!procExecutor.isFinished(procId) && procExecutor.isRunning()) {
+      Threads.sleepWithoutInterrupt(250);
+    }
+  }
+
+  public static <TEnv> void waitNoProcedureRunning(ProcedureExecutor<TEnv> procExecutor) {
+    int stableRuns = 0;
+    while (stableRuns < 10) {
+      if (procExecutor.getActiveExecutorCount() > 0 || procExecutor.getRunnableSet().size() > 0) {
+        stableRuns = 0;
+        Threads.sleepWithoutInterrupt(100);
+      } else {
+        stableRuns++;
+        Threads.sleepWithoutInterrupt(25);
+      }
+    }
+  }
+
+  public static <TEnv> void assertProcNotYetCompleted(ProcedureExecutor<TEnv> procExecutor,
+      long procId) {
+    assertFalse("expected a running proc", procExecutor.isFinished(procId));
+    assertEquals(null, procExecutor.getResult(procId));
+  }
+
+  public static <TEnv> void assertProcNotFailed(ProcedureExecutor<TEnv> procExecutor,
+      long procId) {
+    ProcedureResult result = procExecutor.getResult(procId);
+    assertTrue("expected procedure result", result != null);
+    assertProcNotFailed(result);
+  }
+
+  public static void assertProcNotFailed(final ProcedureResult result) {
+    Exception exception = result.getException();
+    String msg = exception != null ? exception.toString() : "no exception found";
+    assertFalse(msg, result.isFailed());
+  }
+
+  public static void assertIsAbortException(final ProcedureResult result) {
+    LOG.info(result.getException());
+    assertEquals(true, result.isFailed());
+    Throwable cause = result.getException().getCause();
+    assertTrue("expected abort exception, got "+ cause,
+        cause instanceof ProcedureAbortedException);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
new file mode 100644
index 0000000..7fe109e
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
@@ -0,0 +1,338 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+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.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Assert;
+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 TestProcedureExecution {
+  private static final Log LOG = LogFactory.getLog(TestProcedureExecution.class);
+
+  private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
+  private static final Procedure NULL_PROC = null;
+
+  private ProcedureExecutor<Void> procExecutor;
+  private ProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor(htu.getConfiguration(), null, procStore);
+    procStore.start(PROCEDURE_EXECUTOR_SLOTS);
+    procExecutor.start(PROCEDURE_EXECUTOR_SLOTS);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  private static class TestProcedureException extends Exception {
+    public TestProcedureException(String msg) { super(msg); }
+  }
+
+  public static class TestSequentialProcedure extends SequentialProcedure<Void> {
+    private final Procedure[] subProcs;
+    private final List<String> state;
+    private final Exception failure;
+    private final String name;
+
+    public TestSequentialProcedure() {
+      throw new UnsupportedOperationException("recovery should not be triggered here");
+    }
+
+    public TestSequentialProcedure(String name, List<String> state, Procedure... subProcs) {
+      this.state = state;
+      this.subProcs = subProcs;
+      this.name = name;
+      this.failure = null;
+    }
+
+    public TestSequentialProcedure(String name, List<String> state, Exception failure) {
+      this.state = state;
+      this.subProcs = null;
+      this.name = name;
+      this.failure = failure;
+    }
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      state.add(name + "-execute");
+      if (failure != null) {
+        setFailure(new RemoteProcedureException(name + "-failure", failure));
+        return null;
+      }
+      return subProcs;
+    }
+
+    @Override
+    protected void rollback(Void env) {
+      state.add(name + "-rollback");
+    }
+
+    @Override
+    protected boolean abort(Void env) {
+      state.add(name + "-abort");
+      return true;
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testBadSubprocList() {
+    List<String> state = new ArrayList<String>();
+    Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
+    Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2, NULL_PROC);
+    Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
+    long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
+
+    // subProc1 has a "null" subprocedure which is catched as InvalidArgument
+    // failed state with 2 execute and 2 rollback
+    LOG.info(state);
+    ProcedureResult result = procExecutor.getResult(rootId);
+    LOG.info(result.getException());
+    assertTrue(state.toString(), result.isFailed());
+    assertTrue(result.getException().toString(),
+      result.getException().getCause() instanceof IllegalArgumentException);
+
+    assertEquals(state.toString(), 4, state.size());
+    assertEquals("rootProc-execute", state.get(0));
+    assertEquals("subProc1-execute", state.get(1));
+    assertEquals("subProc1-rollback", state.get(2));
+    assertEquals("rootProc-rollback", state.get(3));
+  }
+
+  @Test(timeout=30000)
+  public void testSingleSequentialProc() {
+    List<String> state = new ArrayList<String>();
+    Procedure subProc2 = new TestSequentialProcedure("subProc2", state);
+    Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
+    Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
+    long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
+
+    // successful state, with 3 execute
+    LOG.info(state);
+    ProcedureResult result = procExecutor.getResult(rootId);
+    ProcedureTestingUtility.assertProcNotFailed(result);
+    assertEquals(state.toString(), 3, state.size());
+  }
+
+  @Test(timeout=30000)
+  public void testSingleSequentialProcRollback() {
+    List<String> state = new ArrayList<String>();
+    Procedure subProc2 = new TestSequentialProcedure("subProc2", state,
+                                                     new TestProcedureException("fail test"));
+    Procedure subProc1 = new TestSequentialProcedure("subProc1", state, subProc2);
+    Procedure rootProc = new TestSequentialProcedure("rootProc", state, subProc1);
+    long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, rootProc);
+
+    // the 3rd proc fail, rollback after 2 successful execution
+    LOG.info(state);
+    ProcedureResult result = procExecutor.getResult(rootId);
+    LOG.info(result.getException());
+    assertTrue(state.toString(), result.isFailed());
+    assertTrue(result.getException().toString(),
+      result.getException().getCause() instanceof TestProcedureException);
+
+    assertEquals(state.toString(), 6, state.size());
+    assertEquals("rootProc-execute", state.get(0));
+    assertEquals("subProc1-execute", state.get(1));
+    assertEquals("subProc2-execute", state.get(2));
+    assertEquals("subProc2-rollback", state.get(3));
+    assertEquals("subProc1-rollback", state.get(4));
+    assertEquals("rootProc-rollback", state.get(5));
+  }
+
+  public static class TestFaultyRollback extends SequentialProcedure<Void> {
+    private int retries = 0;
+
+    public TestFaultyRollback() { }
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      setFailure("faulty-rollback-test", new TestProcedureException("test faulty rollback"));
+      return null;
+    }
+
+    @Override
+    protected void rollback(Void env) throws IOException {
+      if (++retries < 3) {
+        LOG.info("inject rollback failure " + retries);
+        throw new IOException("injected failure number " + retries);
+      }
+      LOG.info("execute non faulty rollback step retries=" + retries);
+    }
+
+    @Override
+    protected boolean abort(Void env) { return false; }
+  }
+
+  @Test(timeout=30000)
+  public void testRollbackRetriableFailure() {
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, new TestFaultyRollback());
+
+    ProcedureResult result = procExecutor.getResult(procId);
+    LOG.info(result.getException());
+    assertTrue("expected a failure", result.isFailed());
+    assertTrue(result.getException().toString(),
+      result.getException().getCause() instanceof TestProcedureException);
+  }
+
+  public static class TestWaitingProcedure extends SequentialProcedure<Void> {
+    private final List<String> state;
+    private final boolean hasChild;
+    private final String name;
+
+    public TestWaitingProcedure() {
+      throw new UnsupportedOperationException("recovery should not be triggered here");
+    }
+
+    public TestWaitingProcedure(String name, List<String> state, boolean hasChild) {
+      this.hasChild = hasChild;
+      this.state = state;
+      this.name = name;
+    }
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      state.add(name + "-execute");
+      setState(ProcedureState.WAITING_TIMEOUT);
+      return hasChild ? new Procedure[] { new TestWaitChild(name, state) } : null;
+    }
+
+    @Override
+    protected void rollback(Void env) {
+      state.add(name + "-rollback");
+    }
+
+    @Override
+    protected boolean abort(Void env) {
+      state.add(name + "-abort");
+      return true;
+    }
+
+    public static class TestWaitChild extends SequentialProcedure<Void> {
+      private final List<String> state;
+      private final String name;
+
+      public TestWaitChild() {
+        throw new UnsupportedOperationException("recovery should not be triggered here");
+      }
+
+      public TestWaitChild(String name, List<String> state) {
+        this.name = name;
+        this.state = state;
+      }
+
+      @Override
+      protected Procedure[] execute(Void env) {
+        state.add(name + "-child-execute");
+        return null;
+      }
+
+      @Override
+      protected void rollback(Void env) {
+        state.add(name + "-child-rollback");
+      }
+
+      @Override
+      protected boolean abort(Void env) {
+        state.add(name + "-child-abort");
+        return true;
+      }
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testAbortTimeout() {
+    final int PROC_TIMEOUT_MSEC = 2500;
+    List<String> state = new ArrayList<String>();
+    Procedure proc = new TestWaitingProcedure("wproc", state, false);
+    proc.setTimeout(PROC_TIMEOUT_MSEC);
+    long startTime = EnvironmentEdgeManager.currentTime();
+    long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    long execTime = EnvironmentEdgeManager.currentTime() - startTime;
+    LOG.info(state);
+    assertTrue("we didn't wait enough execTime=" + execTime, execTime >= PROC_TIMEOUT_MSEC);
+    ProcedureResult result = procExecutor.getResult(rootId);
+    LOG.info(result.getException());
+    assertTrue(state.toString(), result.isFailed());
+    assertTrue(result.getException().toString(),
+               result.getException().getCause() instanceof TimeoutException);
+    assertEquals(state.toString(), 2, state.size());
+    assertEquals("wproc-execute", state.get(0));
+    assertEquals("wproc-rollback", state.get(1));
+  }
+
+  @Test(timeout=30000)
+  public void testAbortTimeoutWithChildren() {
+    List<String> state = new ArrayList<String>();
+    Procedure proc = new TestWaitingProcedure("wproc", state, true);
+    proc.setTimeout(2500);
+    long rootId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    LOG.info(state);
+    ProcedureResult result = procExecutor.getResult(rootId);
+    LOG.info(result.getException());
+    assertTrue(state.toString(), result.isFailed());
+    assertTrue(result.getException().toString(),
+               result.getException().getCause() instanceof TimeoutException);
+    assertEquals(state.toString(), 4, state.size());
+    assertEquals("wproc-execute", state.get(0));
+    assertEquals("wproc-child-execute", state.get(1));
+    assertEquals("wproc-child-rollback", state.get(2));
+    assertEquals("wproc-rollback", state.get(3));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureFairRunQueues.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureFairRunQueues.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureFairRunQueues.java
new file mode 100644
index 0000000..e36a295
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureFairRunQueues.java
@@ -0,0 +1,155 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({MasterTests.class, SmallTests.class})
+public class TestProcedureFairRunQueues {
+  private static class TestRunQueue implements ProcedureFairRunQueues.FairObject {
+    private final int priority;
+    private final String name;
+
+    private boolean available = true;
+
+    public TestRunQueue(String name, int priority) {
+      this.name = name;
+      this.priority = priority;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+    private void setAvailable(boolean available) {
+      this.available = available;
+    }
+
+    @Override
+    public boolean isAvailable() {
+      return available;
+    }
+
+    @Override
+    public int getPriority() {
+      return priority;
+    }
+  }
+
+  @Test
+  public void testEmptyFairQueues() throws Exception {
+    ProcedureFairRunQueues<String, TestRunQueue> fairq
+      = new ProcedureFairRunQueues<String, TestRunQueue>(1);
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(null, fairq.poll());
+    }
+  }
+
+  @Test
+  public void testFairQueues() throws Exception {
+    ProcedureFairRunQueues<String, TestRunQueue> fairq
+      = new ProcedureFairRunQueues<String, TestRunQueue>(1);
+    TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
+    TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
+    TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
+
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(a, fairq.poll());
+      assertEquals(b, fairq.poll());
+      assertEquals(m, fairq.poll());
+      assertEquals(m, fairq.poll());
+    }
+  }
+
+  @Test
+  public void testFairQueuesNotAvailable() throws Exception {
+    ProcedureFairRunQueues<String, TestRunQueue> fairq
+      = new ProcedureFairRunQueues<String, TestRunQueue>(1);
+    TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
+    TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
+    TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
+
+    // m is not available
+    m.setAvailable(false);
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(a, fairq.poll());
+      assertEquals(b, fairq.poll());
+    }
+
+    // m is available
+    m.setAvailable(true);
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(m, fairq.poll());
+      assertEquals(m, fairq.poll());
+      assertEquals(a, fairq.poll());
+      assertEquals(b, fairq.poll());
+    }
+
+    // b is not available
+    b.setAvailable(false);
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(m, fairq.poll());
+      assertEquals(m, fairq.poll());
+      assertEquals(a, fairq.poll());
+    }
+
+    assertEquals(m, fairq.poll());
+    m.setAvailable(false);
+    // m should be fetched next, but is no longer available
+    assertEquals(a, fairq.poll());
+    assertEquals(a, fairq.poll());
+    b.setAvailable(true);
+    for (int i = 0; i < 3; ++i) {
+      assertEquals(b, fairq.poll());
+      assertEquals(a, fairq.poll());
+    }
+  }
+
+  @Test
+  public void testFairQueuesDelete() throws Exception {
+    ProcedureFairRunQueues<String, TestRunQueue> fairq
+      = new ProcedureFairRunQueues<String, TestRunQueue>(1);
+    TestRunQueue a = fairq.add("A", new TestRunQueue("A", 1));
+    TestRunQueue b = fairq.add("B", new TestRunQueue("B", 1));
+    TestRunQueue m = fairq.add("M", new TestRunQueue("M", 2));
+
+    // Fetch A and then remove it
+    assertEquals(a, fairq.poll());
+    assertEquals(a, fairq.remove("A"));
+
+    // Fetch B and then remove it
+    assertEquals(b, fairq.poll());
+    assertEquals(b, fairq.remove("B"));
+
+    // Fetch M and then remove it
+    assertEquals(m, fairq.poll());
+    assertEquals(m, fairq.remove("M"));
+
+    // nothing left
+    assertEquals(null, fairq.poll());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
new file mode 100644
index 0000000..0b7395b
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureRecovery.java
@@ -0,0 +1,488 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Assert;
+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 TestProcedureRecovery {
+  private static final Log LOG = LogFactory.getLog(TestProcedureRecovery.class);
+
+  private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
+  private static final Procedure NULL_PROC = null;
+
+  private static ProcedureExecutor<Void> procExecutor;
+  private static ProcedureStore procStore;
+  private static int procSleepInterval;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procStore = ProcedureTestingUtility.createStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor(htu.getConfiguration(), null, procStore);
+    procExecutor.testing = new ProcedureExecutor.Testing();
+    procStore.start(PROCEDURE_EXECUTOR_SLOTS);
+    procExecutor.start(PROCEDURE_EXECUTOR_SLOTS);
+    procSleepInterval = 0;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  private void restart() throws Exception {
+    dumpLogDirState();
+    ProcedureTestingUtility.restart(procExecutor);
+    dumpLogDirState();
+  }
+
+  public static class TestSingleStepProcedure extends SequentialProcedure<Void> {
+    private int step = 0;
+
+    public TestSingleStepProcedure() { }
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      LOG.debug("execute procedure " + this + " step=" + step);
+      step++;
+      setResult(Bytes.toBytes(step));
+      return null;
+    }
+
+    @Override
+    protected void rollback(Void env) { }
+
+    @Override
+    protected boolean abort(Void env) { return true; }
+  }
+
+  public static class BaseTestStepProcedure extends SequentialProcedure<Void> {
+    private AtomicBoolean abort = new AtomicBoolean(false);
+    private int step = 0;
+
+    @Override
+    protected Procedure[] execute(Void env) {
+      LOG.debug("execute procedure " + this + " step=" + step);
+      ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
+      step++;
+      Threads.sleepWithoutInterrupt(procSleepInterval);
+      if (isAborted()) {
+        setFailure(new RemoteProcedureException(getClass().getName(),
+          new ProcedureAbortedException(
+            "got an abort at " + getClass().getName() + " step=" + step)));
+        return null;
+      }
+      return null;
+    }
+
+    @Override
+    protected void rollback(Void env) {
+      LOG.debug("rollback procedure " + this + " step=" + step);
+      ProcedureTestingUtility.toggleKillBeforeStoreUpdate(procExecutor);
+      step++;
+    }
+
+    @Override
+    protected boolean abort(Void env) {
+      abort.set(true);
+      return true;
+    }
+
+    private boolean isAborted() {
+      boolean aborted = abort.get();
+      BaseTestStepProcedure proc = this;
+      while (proc.hasParent() && !aborted) {
+        proc = (BaseTestStepProcedure)procExecutor.getProcedure(proc.getParentProcId());
+        aborted = proc.isAborted();
+      }
+      return aborted;
+    }
+  }
+
+  public static class TestMultiStepProcedure extends BaseTestStepProcedure {
+    public TestMultiStepProcedure() { }
+
+    @Override
+    public Procedure[] execute(Void env) {
+      super.execute(env);
+      return isFailed() ? null : new Procedure[] { new Step1Procedure() };
+    }
+
+    public static class Step1Procedure extends BaseTestStepProcedure {
+      public Step1Procedure() { }
+
+      @Override
+      protected Procedure[] execute(Void env) {
+        super.execute(env);
+        return isFailed() ? null : new Procedure[] { new Step2Procedure() };
+      }
+    }
+
+    public static class Step2Procedure extends BaseTestStepProcedure {
+      public Step2Procedure() { }
+    }
+  }
+
+  @Test
+  public void testNoopLoad() throws Exception {
+    restart();
+  }
+
+  @Test(timeout=30000)
+  public void testSingleStepProcRecovery() throws Exception {
+    Procedure proc = new TestSingleStepProcedure();
+    procExecutor.testing.killBeforeStoreUpdate = true;
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertFalse(procExecutor.isRunning());
+    procExecutor.testing.killBeforeStoreUpdate = false;
+
+    // Restart and verify that the procedures restart
+    long restartTs = EnvironmentEdgeManager.currentTime();
+    restart();
+    waitProcedure(procId);
+    ProcedureResult result = procExecutor.getResult(procId);
+    assertTrue(result.getLastUpdate() > restartTs);
+    ProcedureTestingUtility.assertProcNotFailed(result);
+    assertEquals(1, Bytes.toInt(result.getResult()));
+    long resultTs = result.getLastUpdate();
+
+    // Verify that after another restart the result is still there
+    restart();
+    result = procExecutor.getResult(procId);
+    ProcedureTestingUtility.assertProcNotFailed(result);
+    assertEquals(resultTs, result.getLastUpdate());
+    assertEquals(1, Bytes.toInt(result.getResult()));
+  }
+
+  @Test(timeout=30000)
+  public void testMultiStepProcRecovery() throws Exception {
+    // Step 0 - kill
+    Procedure proc = new TestMultiStepProcedure();
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 0 exec && Step 1 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 1 exec && step 2 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 2 exec
+    restart();
+    waitProcedure(procId);
+    assertTrue(procExecutor.isRunning());
+
+    // The procedure is completed
+    ProcedureResult result = procExecutor.getResult(procId);
+    ProcedureTestingUtility.assertProcNotFailed(result);
+  }
+
+  @Test(timeout=30000)
+  public void testMultiStepRollbackRecovery() throws Exception {
+    // Step 0 - kill
+    Procedure proc = new TestMultiStepProcedure();
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 0 exec && Step 1 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 1 exec && step 2 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 2 exec - rollback - kill
+    procSleepInterval = 2500;
+    restart();
+    assertTrue(procExecutor.abort(procId));
+    waitProcedure(procId);
+    assertFalse(procExecutor.isRunning());
+
+    // rollback - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // rollback - complete
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Restart the executor and get the result
+    restart();
+    waitProcedure(procId);
+
+    // The procedure is completed
+    ProcedureResult result = procExecutor.getResult(procId);
+    ProcedureTestingUtility.assertIsAbortException(result);
+  }
+
+  public static class TestStateMachineProcedure
+      extends StateMachineProcedure<Void, TestStateMachineProcedure.State> {
+    enum State { STATE_1, STATE_2, STATE_3, DONE }
+
+    public TestStateMachineProcedure() {}
+
+    private AtomicBoolean aborted = new AtomicBoolean(false);
+    private int iResult = 0;
+
+    @Override
+    protected StateMachineProcedure.Flow executeFromState(Void env, State state) {
+      switch (state) {
+        case STATE_1:
+          LOG.info("execute step 1 " + this);
+          setNextState(State.STATE_2);
+          iResult += 3;
+          break;
+        case STATE_2:
+          LOG.info("execute step 2 " + this);
+          setNextState(State.STATE_3);
+          iResult += 5;
+          break;
+        case STATE_3:
+          LOG.info("execute step 3 " + this);
+          Threads.sleepWithoutInterrupt(procSleepInterval);
+          if (aborted.get()) {
+            LOG.info("aborted step 3 " + this);
+            setAbortFailure("test", "aborted");
+            break;
+          }
+          setNextState(State.DONE);
+          iResult += 7;
+          setResult(Bytes.toBytes(iResult));
+          return Flow.NO_MORE_STATE;
+        default:
+          throw new UnsupportedOperationException();
+      }
+      return Flow.HAS_MORE_STATE;
+    }
+
+    @Override
+    protected void rollbackState(Void env, final State state) {
+      switch (state) {
+        case STATE_1:
+          LOG.info("rollback step 1 " + this);
+          break;
+        case STATE_2:
+          LOG.info("rollback step 2 " + this);
+          break;
+        case STATE_3:
+          LOG.info("rollback step 3 " + this);
+          break;
+        default:
+          throw new UnsupportedOperationException();
+      }
+    }
+
+    @Override
+    protected State getState(final int stateId) {
+      return State.values()[stateId];
+    }
+
+    @Override
+    protected int getStateId(final State state) {
+      return state.ordinal();
+    }
+
+    @Override
+    protected State getInitialState() {
+      return State.STATE_1;
+    }
+
+    @Override
+    protected boolean abort(Void env) {
+      aborted.set(true);
+      return true;
+    }
+
+    @Override
+    protected void serializeStateData(final OutputStream stream) throws IOException {
+      super.serializeStateData(stream);
+      stream.write(Bytes.toBytes(iResult));
+    }
+
+    @Override
+    protected void deserializeStateData(final InputStream stream) throws IOException {
+      super.deserializeStateData(stream);
+      byte[] data = new byte[4];
+      stream.read(data);
+      iResult = Bytes.toInt(data);
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testStateMachineRecovery() throws Exception {
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
+
+    // Step 1 - kill
+    Procedure proc = new TestStateMachineProcedure();
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 1 exec && Step 2 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 2 exec && step 3 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 3 exec
+    restart();
+    waitProcedure(procId);
+    assertTrue(procExecutor.isRunning());
+
+    // The procedure is completed
+    ProcedureResult result = procExecutor.getResult(procId);
+    ProcedureTestingUtility.assertProcNotFailed(result);
+    assertEquals(15, Bytes.toInt(result.getResult()));
+  }
+
+  @Test(timeout=30000)
+  public void testStateMachineRollbackRecovery() throws Exception {
+    ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExecutor, true);
+    ProcedureTestingUtility.setKillBeforeStoreUpdate(procExecutor, true);
+
+    // Step 1 - kill
+    Procedure proc = new TestStateMachineProcedure();
+    long procId = ProcedureTestingUtility.submitAndWait(procExecutor, proc);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 1 exec && Step 2 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 2 exec && step 3 - kill
+    restart();
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Step 3 exec - rollback step 3 - kill
+    procSleepInterval = 2500;
+    restart();
+    assertTrue(procExecutor.abort(procId));
+    waitProcedure(procId);
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+    assertFalse(procExecutor.isRunning());
+
+    // Rollback step 3 - rollback step 2 - kill
+    restart();
+    waitProcedure(procId);
+    assertFalse(procExecutor.isRunning());
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+
+    // Rollback step 2 - step 1 - kill
+    restart();
+    waitProcedure(procId);
+    assertFalse(procExecutor.isRunning());
+    ProcedureTestingUtility.assertProcNotYetCompleted(procExecutor, procId);
+
+    // Rollback step 1 - complete
+    restart();
+    waitProcedure(procId);
+    assertTrue(procExecutor.isRunning());
+
+    // The procedure is completed
+    ProcedureResult result = procExecutor.getResult(procId);
+    ProcedureTestingUtility.assertIsAbortException(result);
+  }
+
+  private void waitProcedure(final long procId) {
+    ProcedureTestingUtility.waitProcedure(procExecutor, procId);
+    dumpLogDirState();
+  }
+
+  private void dumpLogDirState() {
+    try {
+      FileStatus[] files = fs.listStatus(logDir);
+      if (files != null && files.length > 0) {
+        for (FileStatus file: files) {
+          assertTrue(file.toString(), file.isFile());
+          LOG.debug("log file " + file.getPath() + " size=" + file.getLen());
+        }
+      } else {
+        LOG.debug("no files under: " + logDir);
+      }
+    } catch (IOException e) {
+      LOG.warn("Unable to dump " + logDir, e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
new file mode 100644
index 0000000..88645ed
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureReplayOrder.java
@@ -0,0 +1,226 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+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.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+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, LargeTests.class})
+public class TestProcedureReplayOrder {
+  private static final Log LOG = LogFactory.getLog(TestProcedureReplayOrder.class);
+
+  private static final Procedure NULL_PROC = null;
+
+  private ProcedureExecutor<Void> procExecutor;
+  private TestProcedureEnv procEnv;
+  private ProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    htu.getConfiguration().setInt("hbase.procedure.store.wal.sync.wait.msec", 10);
+
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procEnv = new TestProcedureEnv();
+    procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
+    procExecutor = new ProcedureExecutor(htu.getConfiguration(), procEnv, procStore);
+    procStore.start(24);
+    procExecutor.start(1);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procExecutor.stop();
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  @Test(timeout=90000)
+  public void testSingleStepReplyOrder() throws Exception {
+    // avoid the procedure to be runnable
+    procEnv.setAcquireLock(false);
+
+    // submit the procedures
+    submitProcedures(16, 25, TestSingleStepProcedure.class);
+
+    // restart the executor and allow the procedures to run
+    ProcedureTestingUtility.restart(procExecutor, new Runnable() {
+      @Override
+      public void run() {
+        procEnv.setAcquireLock(true);
+      }
+    });
+
+    // wait the execution of all the procedures and
+    // assert that the execution order was sorted by procId
+    ProcedureTestingUtility.waitNoProcedureRunning(procExecutor);
+    procEnv.assertSortedExecList();
+
+    // TODO: FIXME: This should be revisited
+  }
+
+  @Ignore
+  @Test(timeout=90000)
+  public void testMultiStepReplyOrder() throws Exception {
+    // avoid the procedure to be runnable
+    procEnv.setAcquireLock(false);
+
+    // submit the procedures
+    submitProcedures(16, 10, TestTwoStepProcedure.class);
+
+    // restart the executor and allow the procedures to run
+    ProcedureTestingUtility.restart(procExecutor, new Runnable() {
+      @Override
+      public void run() {
+        procEnv.setAcquireLock(true);
+      }
+    });
+
+    fail("TODO: FIXME: NOT IMPLEMENT REPLAY ORDER");
+  }
+
+  private void submitProcedures(final int nthreads, final int nprocPerThread,
+      final Class<?> procClazz) throws Exception {
+    Thread[] submitThreads = new Thread[nthreads];
+    for (int i = 0; i < submitThreads.length; ++i) {
+      submitThreads[i] = new Thread() {
+        @Override
+        public void run() {
+          for (int i = 0; i < nprocPerThread; ++i) {
+            try {
+              procExecutor.submitProcedure((Procedure)procClazz.newInstance());
+            } catch (InstantiationException|IllegalAccessException e) {
+              LOG.error("unable to instantiate the procedure", e);
+              fail("failure during the proc.newInstance(): " + e.getMessage());
+            }
+          }
+        }
+      };
+    }
+
+    for (int i = 0; i < submitThreads.length; ++i) {
+      submitThreads[i].start();
+    }
+
+    for (int i = 0; i < submitThreads.length; ++i) {
+      submitThreads[i].join();
+    }
+  }
+
+  private static class TestProcedureEnv {
+    private ArrayList<Long> execList = new ArrayList<Long>();
+    private boolean acquireLock = true;
+
+    public void setAcquireLock(boolean acquireLock) {
+      this.acquireLock = acquireLock;
+    }
+
+    public boolean canAcquireLock() {
+      return acquireLock;
+    }
+
+    public void addToExecList(final Procedure proc) {
+      execList.add(proc.getProcId());
+    }
+
+    public ArrayList<Long> getExecList() {
+      return execList;
+    }
+
+    public void assertSortedExecList() {
+      LOG.debug("EXEC LIST: " + execList);
+      for (int i = 1; i < execList.size(); ++i) {
+        assertTrue("exec list not sorted: " + execList.get(i-1) + " >= " + execList.get(i),
+          execList.get(i-1) < execList.get(i));
+      }
+    }
+  }
+
+  public static class TestSingleStepProcedure extends SequentialProcedure<TestProcedureEnv> {
+    public TestSingleStepProcedure() { }
+
+    @Override
+    protected Procedure[] execute(TestProcedureEnv env) {
+      LOG.debug("execute procedure " + this);
+      env.addToExecList(this);
+      return null;
+    }
+
+    protected boolean acquireLock(final TestProcedureEnv env) {
+      return env.canAcquireLock();
+    }
+
+    @Override
+    protected void rollback(TestProcedureEnv env) { }
+
+    @Override
+    protected boolean abort(TestProcedureEnv env) { return true; }
+  }
+
+  public static class TestTwoStepProcedure extends SequentialProcedure<TestProcedureEnv> {
+    public TestTwoStepProcedure() { }
+
+    @Override
+    protected Procedure[] execute(TestProcedureEnv env) {
+      LOG.debug("execute procedure " + this);
+      env.addToExecList(this);
+      return new Procedure[] { new TestSingleStepProcedure() };
+    }
+
+    protected boolean acquireLock(final TestProcedureEnv env) {
+      return true;
+    }
+
+    @Override
+    protected void rollback(TestProcedureEnv env) { }
+
+    @Override
+    protected boolean abort(TestProcedureEnv env) { return true; }
+  }
+}
\ No newline at end of file


[19/28] 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/7af69304/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>

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

Branch: refs/heads/hbase-12439
Commit: 7af69304b1b11a4e63d28dd5ded3d956f32df3c1
Parents: 9d684ac
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:01:20 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:05 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 +-
 .../master/procedure/DeleteTableProcedure.java  |    1 +
 .../hadoop/hbase/master/TestCatalogJanitor.java |    7 +-
 10 files changed, 2921 insertions(+), 554 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7af69304/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/7af69304/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/7af69304/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


[09/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-protocol/src/main/protobuf/Procedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Procedure.proto b/hbase-protocol/src/main/protobuf/Procedure.proto
new file mode 100644
index 0000000..232c290
--- /dev/null
+++ b/hbase-protocol/src/main/protobuf/Procedure.proto
@@ -0,0 +1,114 @@
+/**
+ * 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 = "ProcedureProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "ErrorHandling.proto";
+
+enum ProcedureState {
+  INITIALIZING = 1;         // Procedure in construction, not yet added to the executor
+  RUNNABLE = 2;             // Procedure added to the executor, and ready to be executed
+  WAITING = 3;              // The procedure is waiting on children to be completed
+  WAITING_TIMEOUT = 4;      // The procedure is waiting a timout or an external event
+  ROLLEDBACK = 5;           // The procedure failed and was rolledback
+  FINISHED = 6;             // The procedure execution is completed. may need a rollback if failed.
+}
+
+/**
+ * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
+ */
+message Procedure {
+  // internal "static" state
+  required string class_name = 1;        // full classname to be able to instantiate the procedure
+  optional uint64 parent_id = 2;         // parent if not a root-procedure otherwise not set
+  required uint64 proc_id = 3;
+  required uint64 start_time = 4;
+  optional string owner = 5;
+
+  // internal "runtime" state
+  required ProcedureState state = 6;
+  repeated uint32 stack_id = 7;          // stack indices in case the procedure was running
+  required uint64 last_update = 8;
+  optional uint32 timeout = 9;
+
+  // user state/results
+  optional ForeignExceptionMessage exception = 10;
+  optional bytes result = 11;           // opaque (user) result structure
+  optional bytes state_data = 12;       // opaque (user) procedure internal-state
+}
+
+/**
+ * SequentialProcedure data
+ */
+message SequentialProcedureData {
+  required bool executed = 1;
+}
+
+/**
+ * StateMachineProcedure data
+ */
+message StateMachineProcedureData {
+  repeated uint32 state = 1;
+}
+
+/**
+ * Procedure WAL header
+ */
+message ProcedureWALHeader {
+  required uint32 version = 1;
+  required uint32 type = 2;
+  required uint64 log_id = 3;
+  required uint64 min_proc_id = 4;
+}
+
+/**
+ * Procedure WAL trailer
+ */
+message ProcedureWALTrailer {
+  required uint32 version = 1;
+  required uint64 tracker_pos = 2;
+}
+
+message ProcedureStoreTracker {
+  message TrackerNode {
+    required uint64 start_id = 1;
+    repeated uint64 updated = 2;
+    repeated uint64 deleted = 3;
+  }
+
+  repeated TrackerNode node = 1;
+}
+
+message ProcedureWALEntry {
+  enum Type {
+    EOF     = 1;
+    INIT    = 2;
+    INSERT  = 3;
+    UPDATE  = 4;
+    DELETE  = 5;
+    COMPACT = 6;
+  }
+
+  required Type type = 1;
+  repeated Procedure procedure = 2;
+  optional uint64 proc_id = 3;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f7ae209..ce49c04 100644
--- a/pom.xml
+++ b/pom.xml
@@ -56,6 +56,7 @@
     <module>hbase-client</module>
     <module>hbase-hadoop-compat</module>
     <module>hbase-common</module>
+    <module>hbase-procedure</module>
     <module>hbase-it</module>
     <module>hbase-examples</module>
     <module>hbase-prefix-tree</module>
@@ -871,7 +872,7 @@
                 </fileMapper>
               </fileMappers>
               <outputDir>${basedir}/target/asciidoc</outputDir>
-            </transformationSet> 
+            </transformationSet>
           </transformationSets>
         </configuration>
       </plugin>
@@ -1014,7 +1015,7 @@
       <plugin>
         <groupId>org.asciidoctor</groupId>
         <artifactId>asciidoctor-maven-plugin</artifactId>
-        <version>1.5.2</version> 
+        <version>1.5.2</version>
         <inherited>false</inherited>
         <dependencies>
           <dependency>
@@ -1034,10 +1035,10 @@
         </configuration>
         <executions>
           <execution>
-            <id>output-html</id> 
+            <id>output-html</id>
             <phase>site</phase>
             <goals>
-              <goal>process-asciidoc</goal> 
+              <goal>process-asciidoc</goal>
             </goals>
             <configuration>
               <attributes>
@@ -1191,6 +1192,7 @@
       Modules are pretty heavy-weight things, so doing this work isn't too bad. -->
     <server.test.jar>hbase-server-${project.version}-tests.jar</server.test.jar>
     <common.test.jar>hbase-common-${project.version}-tests.jar</common.test.jar>
+    <procedure.test.jar>hbase-procedure-${project.version}-tests.jar</procedure.test.jar>
     <it.test.jar>hbase-it-${project.version}-tests.jar</it.test.jar>
     <annotations.test.jar>hbase-annotations-${project.version}-tests.jar</annotations.test.jar>
     <surefire.version>2.18</surefire.version>
@@ -1262,6 +1264,17 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-procedure</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-procedure</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-hadoop-compat</artifactId>
         <version>${project.version}</version>
       </dependency>


[22/28] 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/c5bf41a7/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 97d1af6..a07516d 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -89,3 +89,49 @@ message DeleteTableStateData {
   required TableName table_name = 2;
   repeated RegionInfo region_info = 3;
 }
+
+enum AddColumnFamilyState {
+  ADD_COLUMN_FAMILY_PREPARE = 1;
+  ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
+  ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  ADD_COLUMN_FAMILY_POST_OPERATION = 4;
+  ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message AddColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum ModifyColumnFamilyState {
+  MODIFY_COLUMN_FAMILY_PREPARE = 1;
+  MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
+  MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
+  MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
+}
+
+message ModifyColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required ColumnFamilySchema columnfamily_schema = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}
+
+enum DeleteColumnFamilyState {
+  DELETE_COLUMN_FAMILY_PREPARE = 1;
+  DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
+  DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
+  DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
+  DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
+  DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
+}
+
+message DeleteColumnFamilyStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bytes columnfamily_name = 3;
+  optional TableSchema unmodified_table_schema = 4;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5bf41a7/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/c5bf41a7/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/c5bf41a7/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..98a00c2
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java
@@ -0,0 +1,409 @@
+/**
+ * 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 final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private TableName tableName;
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private HColumnDescriptor cfDescriptor;
+  private UserGroupInformation user;
+
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled;
+
+  public AddColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  public AddColumnFamilyProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final HColumnDescriptor cfDescriptor) throws IOException {
+    this.tableName = tableName;
+    this.cfDescriptor = cfDescriptor;
+    this.user = env.getRequestUser().getUGI();
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final AddColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case ADD_COLUMN_FAMILY_PREPARE:
+        prepareAdd(env);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case ADD_COLUMN_FAMILY_PRE_OPERATION:
+        preAdd(env, state);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case ADD_COLUMN_FAMILY_POST_OPERATION:
+        postAdd(env, state);
+        setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.warn("Error trying to add the column family" + getColumnFamilyName() + " to the table "
+          + tableName + " (in state=" + state + ")", e);
+
+      setFailure("master-add-columnfamily", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final AddColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case ADD_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case ADD_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case ADD_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for adding the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected AddColumnFamilyState getState(final int stateId) {
+    return AddColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final AddColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected AddColumnFamilyState getInitialState() {
+    return AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(AddColumnFamilyState state) {
+    if (aborted.get()) {
+      setAbortFailure("add-columnfamily", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      tableName,
+      EventType.C_M_ADD_FAMILY.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(tableName);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.AddColumnFamilyStateData.Builder addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilySchema(cfDescriptor.convert());
+    if (unmodifiedHTableDescriptor != null) {
+      addCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+
+    addCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
+        MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
+    cfDescriptor = HColumnDescriptor.convert(addCFMsg.getColumnfamilySchema());
+    if (addCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(addCFMsg.getUnmodifiedTableSchema());
+    }
+  }
+
+  @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");
+    }
+  }
+
+  /**
+   * 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,
+        getRegionInfoList(env), 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(), getRegionInfoList(env))) {
+      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;
+        }
+      });
+    }
+  }
+
+  private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+    if (regionInfoList == null) {
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+    }
+    return regionInfoList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5bf41a7/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..a053c89
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java
@@ -0,0 +1,441 @@
+/**
+ * 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 final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private TableName tableName;
+  private byte [] familyName;
+  private UserGroupInformation user;
+
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled;
+
+  public DeleteColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  public DeleteColumnFamilyProcedure(
+      final MasterProcedureEnv env,
+      final TableName tableName,
+      final byte[] familyName) throws IOException {
+    this.tableName = tableName;
+    this.familyName = familyName;
+    this.user = env.getRequestUser().getUGI();
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, DeleteColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case DELETE_COLUMN_FAMILY_PREPARE:
+        prepareDelete(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case DELETE_COLUMN_FAMILY_PRE_OPERATION:
+        preDelete(env, state);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT);
+        break;
+      case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
+        deleteFromFs(env);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case DELETE_COLUMN_FAMILY_POST_OPERATION:
+        postDelete(env, state);
+        setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn("Error trying to delete the column family " + getColumnFamilyName()
+          + " from table " + tableName + "(in state=" + state + ")", e);
+      } else {
+        LOG.error("Error trying to delete the column family " + getColumnFamilyName()
+          + " from table " + tableName + "(in state=" + state + ")", e);
+        setFailure("master-delete-column-family", e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case DELETE_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
+        // Once we reach to this state - we could NOT rollback - as it is tricky to undelete
+        // the deleted files. We are not suppose to reach here, throw exception so that we know
+        // there is a code bug to investigate.
+        throw new UnsupportedOperationException(this + " rollback of state=" + state
+            + " is unsupported.");
+      case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case DELETE_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case DELETE_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for deleting the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected DeleteColumnFamilyState getState(final int stateId) {
+    return DeleteColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final DeleteColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected DeleteColumnFamilyState getInitialState() {
+    return DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(DeleteColumnFamilyState state) {
+    if (aborted.get() && isRollbackSupported(state)) {
+      setAbortFailure("delete-columnfamily", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      tableName,
+      EventType.C_M_DELETE_FAMILY.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(tableName);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.DeleteColumnFamilyStateData.Builder deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilyName(ByteStringer.wrap(familyName));
+    if (unmodifiedHTableDescriptor != null) {
+      deleteCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+
+    deleteCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+    MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
+        MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
+    familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
+
+    if (deleteCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(deleteCFMsg.getUnmodifiedTableSchema());
+    }
+  }
+
+  @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");
+    }
+  }
+
+  /**
+   * 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,
+      getRegionInfoList(env), 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(), getRegionInfoList(env))) {
+      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;
+  }
+
+  private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+    if (regionInfoList == null) {
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+    }
+    return regionInfoList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5bf41a7/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..138ebd8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java
@@ -0,0 +1,384 @@
+/**
+ * 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 final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private TableName tableName;
+  private HTableDescriptor unmodifiedHTableDescriptor;
+  private HColumnDescriptor cfDescriptor;
+  private UserGroupInformation user;
+
+  private Boolean traceEnabled;
+
+  public ModifyColumnFamilyProcedure() {
+    this.unmodifiedHTableDescriptor = 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.traceEnabled = null;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env,
+      final ModifyColumnFamilyState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case MODIFY_COLUMN_FAMILY_PREPARE:
+        prepareModify(env);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PRE_OPERATION);
+        break;
+      case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
+        preModify(env, state);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_POST_OPERATION);
+        break;
+      case MODIFY_COLUMN_FAMILY_POST_OPERATION:
+        postModify(env, state);
+        setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
+        break;
+      case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      LOG.warn("Error trying to modify the column family " + getColumnFamilyName()
+          + " of the table " + tableName + "(in state=" + state + ")", e);
+
+      setFailure("master-modify-columnfamily", e);
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case MODIFY_COLUMN_FAMILY_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to undo?
+        break;
+      case MODIFY_COLUMN_FAMILY_PREPARE:
+        break; // nothing to do
+      default:
+        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      // This will be retried. Unless there is a bug in the code,
+      // this should be just a "temporary error" (e.g. network down)
+      LOG.warn("Failed rollback attempt step " + state + " for adding the column family"
+          + getColumnFamilyName() + " to the table " + tableName, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected ModifyColumnFamilyState getState(final int stateId) {
+    return ModifyColumnFamilyState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final ModifyColumnFamilyState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyColumnFamilyState getInitialState() {
+    return ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(ModifyColumnFamilyState state) {
+    if (aborted.get()) {
+      setAbortFailure("modify-columnfamily", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      tableName,
+      EventType.C_M_MODIFY_FAMILY.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(tableName);
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.ModifyColumnFamilyStateData.Builder modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setTableName(ProtobufUtil.toProtoTableName(tableName))
+            .setColumnfamilySchema(cfDescriptor.convert());
+    if (unmodifiedHTableDescriptor != null) {
+      modifyCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+
+    modifyCFMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
+        MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo());
+    tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
+    cfDescriptor = HColumnDescriptor.convert(modifyCFMsg.getColumnfamilySchema());
+    if (modifyCFMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor = HTableDescriptor.convert(modifyCFMsg.getUnmodifiedTableSchema());
+    }
+  }
+
+  @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");
+    }
+  }
+
+  /**
+   * 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;
+    }
+
+    List<HRegionInfo> regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+    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/c5bf41a7/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/c5bf41a7/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/c5bf41a7/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/c5bf41a7/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/c5bf41a7/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();
+  }
+}


[13/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java
new file mode 100644
index 0000000..0aebd5a
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureResult.java
@@ -0,0 +1,95 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Once a Procedure completes the ProcedureExecutor takes all the useful
+ * information of the procedure (e.g. exception/result) and creates a ProcedureResult.
+ * The user of the Procedure framework will get the procedure result with
+ * procedureExecutor.getResult(procId)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ProcedureResult {
+  private final RemoteProcedureException exception;
+  private final long lastUpdate;
+  private final long startTime;
+  private final byte[] result;
+
+  private long clientAckTime = -1;
+
+  public ProcedureResult(final long startTime, final long lastUpdate,
+      final RemoteProcedureException exception) {
+    this.lastUpdate = lastUpdate;
+    this.startTime = startTime;
+    this.exception = exception;
+    this.result = null;
+  }
+
+  public ProcedureResult(final long startTime, final long lastUpdate, final byte[] result) {
+    this.lastUpdate = lastUpdate;
+    this.startTime = startTime;
+    this.exception = null;
+    this.result = result;
+  }
+
+  public boolean isFailed() {
+    return exception != null;
+  }
+
+  public RemoteProcedureException getException() {
+    return exception;
+  }
+
+  public boolean hasResultData() {
+    return result != null;
+  }
+
+  public byte[] getResult() {
+    return result;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public long getLastUpdate() {
+    return lastUpdate;
+  }
+
+  public long executionTime() {
+    return lastUpdate - startTime;
+  }
+
+  public boolean hasClientAckTime() {
+    return clientAckTime > 0;
+  }
+
+  public long getClientAckTime() {
+    return clientAckTime;
+  }
+
+  @InterfaceAudience.Private
+  protected void setClientAckTime(final long timestamp) {
+    this.clientAckTime = timestamp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureRunnableSet.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureRunnableSet.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureRunnableSet.java
new file mode 100644
index 0000000..2d7ba39
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureRunnableSet.java
@@ -0,0 +1,78 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Keep track of the runnable procedures
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ProcedureRunnableSet {
+  /**
+   * Inserts the specified element at the front of this queue.
+   * @param proc the Procedure to add
+   */
+  void addFront(Procedure proc);
+
+  /**
+   * Inserts the specified element at the end of this queue.
+   * @param proc the Procedure to add
+   */
+  void addBack(Procedure proc);
+
+  /**
+   * The procedure can't run at the moment.
+   * add it back to the queue, giving priority to someone else.
+   * @param proc the Procedure to add back to the list
+   */
+  void yield(Procedure proc);
+
+  /**
+   * The procedure in execution completed.
+   * This can be implemented to perform cleanups.
+   * @param proc the Procedure that completed the execution.
+   */
+  void completionCleanup(Procedure proc);
+
+  /**
+   * Fetch one Procedure from the queue
+   * @return the Procedure ID to execute, or null if nothing present.
+   */
+  Long poll();
+
+  /**
+   * In case the class is blocking on poll() waiting for items to be added,
+   * this method should awake poll() and poll() should return.
+   */
+  void signalAll();
+
+  /**
+   * Returns the number of elements in this collection.
+   * @return the number of elements in this collection.
+   */
+  int size();
+
+  /**
+   * Removes all of the elements from this collection.
+   */
+  void clear();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSimpleRunQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSimpleRunQueue.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSimpleRunQueue.java
new file mode 100644
index 0000000..7b17fb2
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureSimpleRunQueue.java
@@ -0,0 +1,121 @@
+/**
+ * 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.procedure2;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Simple runqueue for the procedures
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureSimpleRunQueue implements ProcedureRunnableSet {
+  private final Deque<Long> runnables = new ArrayDeque<Long>();
+  private final ReentrantLock lock = new ReentrantLock();
+  private final Condition waitCond = lock.newCondition();
+
+  @Override
+  public void addFront(final Procedure proc) {
+    lock.lock();
+    try {
+      runnables.addFirst(proc.getProcId());
+      waitCond.signal();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void addBack(final Procedure proc) {
+    lock.lock();
+    try {
+      runnables.addLast(proc.getProcId());
+      waitCond.signal();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void yield(final Procedure proc) {
+    addBack(proc);
+  }
+
+  @Override
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
+  public Long poll() {
+    lock.lock();
+    try {
+      if (runnables.isEmpty()) {
+        waitCond.await();
+        if (!runnables.isEmpty()) {
+          return runnables.pop();
+        }
+      } else {
+        return runnables.pop();
+      }
+    } 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 {
+      runnables.clear();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public int size() {
+    lock.lock();
+    try {
+      return runnables.size();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void completionCleanup(Procedure proc) {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
new file mode 100644
index 0000000..177ff5b
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+// TODO: Not used yet
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ProcedureYieldException extends ProcedureException {
+  /** default constructor */
+  public ProcedureYieldException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public ProcedureYieldException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
new file mode 100644
index 0000000..6be512d
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
@@ -0,0 +1,116 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * A RemoteProcedureException is an exception from another thread or process.
+ * <p>
+ * RemoteProcedureExceptions are sent to 'remote' peers to signal an abort in the face of failures.
+ * When serialized for transmission we encode using Protobufs to ensure version compatibility.
+ * <p>
+ * RemoteProcedureException exceptions contain a Throwable as its cause.
+ * This can be a "regular" exception generated locally or a ProxyThrowable that is a representation
+ * of the original exception created on original 'remote' source.  These ProxyThrowables have their
+ * their stacks traces and messages overridden to reflect the original 'remote' exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+@SuppressWarnings("serial")
+public class RemoteProcedureException extends ProcedureException {
+
+  /**
+   * Name of the throwable's source such as a host or thread name.  Must be non-null.
+   */
+  private final String source;
+
+  /**
+   * Create a new RemoteProcedureException that can be serialized.
+   * It is assumed that this came form a local source.
+   * @param source
+   * @param cause
+   */
+  public RemoteProcedureException(String source, Throwable cause) {
+    super(cause);
+    assert source != null;
+    assert cause != null;
+    this.source = source;
+  }
+
+  public String getSource() {
+    return source;
+  }
+
+  public IOException unwrapRemoteException() {
+    if (getCause() instanceof RemoteException) {
+      return ((RemoteException)getCause()).unwrapRemoteException();
+    }
+    if (getCause() instanceof IOException) {
+      return (IOException)getCause();
+    }
+    return new IOException(getCause());
+  }
+
+  @Override
+  public String toString() {
+    String className = getCause().getClass().getName();
+    return className + " via " + getSource() + ":" + getLocalizedMessage();
+  }
+
+  /**
+   * Converts a RemoteProcedureException to an array of bytes.
+   * @param source the name of the external exception source
+   * @param t the "local" external exception (local)
+   * @return protobuf serialized version of RemoteProcedureException
+   */
+  public static byte[] serialize(String source, Throwable t) {
+    return toProto(source, t).toByteArray();
+  }
+
+  /**
+   * Takes a series of bytes and tries to generate an RemoteProcedureException instance for it.
+   * @param bytes
+   * @return the ForeignExcpetion instance
+   * @throws InvalidProtocolBufferException if there was deserialization problem this is thrown.
+   */
+  public static RemoteProcedureException deserialize(byte[] bytes)
+      throws InvalidProtocolBufferException {
+    return fromProto(ForeignExceptionMessage.parseFrom(bytes));
+  }
+
+  public ForeignExceptionMessage convert() {
+    return ForeignExceptionUtil.toProtoForeignException(getSource(), getCause());
+  }
+
+  public static ForeignExceptionMessage toProto(String source, Throwable t) {
+    return ForeignExceptionUtil.toProtoForeignException(source, t);
+  }
+
+  public static RemoteProcedureException fromProto(final ForeignExceptionMessage eem) {
+    return new RemoteProcedureException(eem.getSource(), ForeignExceptionUtil.toIOException(eem));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
new file mode 100644
index 0000000..bc1af20
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
@@ -0,0 +1,185 @@
+/**
+ * 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.procedure2;
+
+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.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+
+/**
+ * Internal state of the ProcedureExecutor that describes the state of a "Root Procedure".
+ * A "Root Procedure" is a Procedure without parent, each subprocedure will be
+ * added to the "Root Procedure" stack (or rollback-stack).
+ *
+ * RootProcedureState is used and managed only by the ProcedureExecutor.
+ *    Long rootProcId = getRootProcedureId(proc);
+ *    rollbackStack.get(rootProcId).acquire(proc)
+ *    rollbackStack.get(rootProcId).release(proc)
+ *    ...
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class RootProcedureState {
+  private static final Log LOG = LogFactory.getLog(RootProcedureState.class);
+
+  private enum State {
+    RUNNING,         // The Procedure is running or ready to run
+    FAILED,          // The Procedure failed, waiting for the rollback executing
+    ROLLINGBACK,     // The Procedure failed and the execution was rolledback
+  }
+
+  private ArrayList<Procedure> subprocedures = null;
+  private State state = State.RUNNING;
+  private int running = 0;
+
+  public synchronized boolean isFailed() {
+    switch (state) {
+      case ROLLINGBACK:
+      case FAILED:
+        return true;
+      default:
+        break;
+    }
+    return false;
+  }
+
+  public synchronized boolean isRollingback() {
+    return state == State.ROLLINGBACK;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to mark rollback execution
+   */
+  protected synchronized boolean setRollback() {
+    if (running == 0 && state == State.FAILED) {
+      state = State.ROLLINGBACK;
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to mark rollback execution
+   */
+  protected synchronized void unsetRollback() {
+    assert state == State.ROLLINGBACK;
+    state = State.FAILED;
+  }
+
+  protected synchronized List<Procedure> getSubprocedures() {
+    return subprocedures;
+  }
+
+  protected synchronized RemoteProcedureException getException() {
+    if (subprocedures != null) {
+      for (Procedure proc: subprocedures) {
+        if (proc.hasException()) {
+          return proc.getException();
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to mark the procedure step as running.
+   */
+  protected synchronized boolean acquire(final Procedure proc) {
+    if (state != State.RUNNING) return false;
+
+    running++;
+    return true;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to mark the procedure step as finished.
+   */
+  protected synchronized void release(final Procedure proc) {
+    running--;
+  }
+
+  protected synchronized void abort() {
+    if (state == State.RUNNING) {
+      state = State.FAILED;
+    }
+  }
+
+  /**
+   * Called by the ProcedureExecutor after the procedure step is completed,
+   * to add the step to the rollback list (or procedure stack)
+   */
+  protected synchronized void addRollbackStep(final Procedure proc) {
+    if (proc.isFailed()) {
+      state = State.FAILED;
+    }
+    if (subprocedures == null) {
+      subprocedures = new ArrayList<Procedure>();
+    }
+    proc.addStackIndex(subprocedures.size());
+    subprocedures.add(proc);
+  }
+
+  /**
+   * Called on store load by the ProcedureExecutor to load part of the stack.
+   *
+   * Each procedure has its own stack-positions. Which means we have to write
+   * to the store only the Procedure we executed, and nothing else.
+   * on load we recreate the full stack by aggregating each procedure stack-positions.
+   */
+  protected synchronized void loadStack(final Procedure proc) {
+    int[] stackIndexes = proc.getStackIndexes();
+    if (stackIndexes != null) {
+      if (subprocedures == null) {
+        subprocedures = new ArrayList<Procedure>();
+      }
+      int diff = (1 + stackIndexes[stackIndexes.length - 1]) - subprocedures.size();
+      if (diff > 0) {
+        subprocedures.ensureCapacity(1 + stackIndexes[stackIndexes.length - 1]);
+        while (diff-- > 0) subprocedures.add(null);
+      }
+      for (int i = 0; i < stackIndexes.length; ++i) {
+        subprocedures.set(stackIndexes[i], proc);
+      }
+    }
+    if (proc.getState() == ProcedureState.ROLLEDBACK) {
+      state = State.ROLLINGBACK;
+    } else if (proc.isFailed()) {
+      state = State.FAILED;
+    }
+  }
+
+  /**
+   * Called on store load by the ProcedureExecutor to validate the procedure stack.
+   */
+  protected synchronized boolean isValid() {
+    if (subprocedures != null) {
+      for (Procedure proc: subprocedures) {
+        if (proc == null) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
new file mode 100644
index 0000000..b4b35f2
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
@@ -0,0 +1,81 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData;
+
+/**
+ * A SequentialProcedure describes one step in a procedure chain.
+ *   -> Step 1 -> Step 2 -> Step 3
+ *
+ * The main difference from a base Procedure is that the execute() of a
+ * SequentialProcedure will be called only once, there will be no second
+ * execute() call once the child are finished. which means once the child
+ * of a SequentialProcedure are completed the SequentialProcedure is completed too.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class SequentialProcedure<TEnvironment> extends Procedure<TEnvironment> {
+  private boolean executed = false;
+
+  @Override
+  protected Procedure[] doExecute(final TEnvironment env)
+      throws ProcedureYieldException {
+    updateTimestamp();
+    try {
+      Procedure[] children = !executed ? execute(env) : null;
+      executed = !executed;
+      return children;
+    } finally {
+      updateTimestamp();
+    }
+  }
+
+  @Override
+  protected void doRollback(final TEnvironment env) throws IOException {
+    updateTimestamp();
+    if (executed) {
+      try {
+        rollback(env);
+        executed = !executed;
+      } finally {
+        updateTimestamp();
+      }
+    }
+  }
+
+  @Override
+  protected void serializeStateData(final OutputStream stream) throws IOException {
+    SequentialProcedureData.Builder data = SequentialProcedureData.newBuilder();
+    data.setExecuted(executed);
+    data.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  protected void deserializeStateData(final InputStream stream) throws IOException {
+    SequentialProcedureData data = SequentialProcedureData.parseDelimitedFrom(stream);
+    executed = data.getExecuted();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
new file mode 100644
index 0000000..eab96e4
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -0,0 +1,166 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.StateMachineProcedureData;
+
+/**
+ * Procedure described by a series of steps.
+ *
+ * The procedure implementor must have an enum of 'states', describing
+ * the various step of the procedure.
+ * Once the procedure is running, the procedure-framework will call executeFromState()
+ * using the 'state' provided by the user. The first call to executeFromState()
+ * will be performed with 'state = null'. The implementor can jump between
+ * states using setNextState(MyStateEnum.ordinal()).
+ * The rollback will call rollbackState() for each state that was executed, in reverse order.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class StateMachineProcedure<TEnvironment, TState>
+    extends Procedure<TEnvironment> {
+  private int stateCount = 0;
+  private int[] states = null;
+
+  protected enum Flow {
+    HAS_MORE_STATE,
+    NO_MORE_STATE,
+  }
+
+  /**
+   * called to perform a single step of the specified 'state' of the procedure
+   * @param state state to execute
+   * @return Flow.NO_MORE_STATE if the procedure is completed,
+   *         Flow.HAS_MORE_STATE if there is another step.
+   */
+  protected abstract Flow executeFromState(TEnvironment env, TState state)
+    throws ProcedureYieldException;
+
+  /**
+   * called to perform the rollback of the specified state
+   * @param state state to rollback
+   * @throws IOException temporary failure, the rollback will retry later
+   */
+  protected abstract void rollbackState(TEnvironment env, TState state)
+    throws IOException;
+
+  /**
+   * Convert an ordinal (or state id) to an Enum (or more descriptive) state object.
+   * @param stateId the ordinal() of the state enum (or state id)
+   * @return the state enum object
+   */
+  protected abstract TState getState(int stateId);
+
+  /**
+   * Convert the Enum (or more descriptive) state object to an ordinal (or state id).
+   * @param state the state enum object
+   * @return stateId the ordinal() of the state enum (or state id)
+   */
+  protected abstract int getStateId(TState state);
+
+  /**
+   * Return the initial state object that will be used for the first call to executeFromState().
+   * @return the initial state enum object
+   */
+  protected abstract TState getInitialState();
+
+  /**
+   * Set the next state for the procedure.
+   * @param state the state enum object
+   */
+  protected void setNextState(final TState state) {
+    setNextState(getStateId(state));
+  }
+
+  @Override
+  protected Procedure[] execute(final TEnvironment env)
+      throws ProcedureYieldException {
+    updateTimestamp();
+    try {
+      TState state = stateCount > 0 ? getState(states[stateCount-1]) : getInitialState();
+      if (stateCount == 0) {
+        setNextState(getStateId(state));
+      }
+      if (executeFromState(env, state) == Flow.NO_MORE_STATE) {
+        // completed
+        return null;
+      }
+      return (isWaiting() || isFailed()) ? null : new Procedure[] {this};
+    } finally {
+      updateTimestamp();
+    }
+  }
+
+  @Override
+  protected void rollback(final TEnvironment env) throws IOException {
+    try {
+      updateTimestamp();
+      rollbackState(env, stateCount > 0 ? getState(states[stateCount-1]) : getInitialState());
+      stateCount--;
+    } finally {
+      updateTimestamp();
+    }
+  }
+
+  /**
+   * Set the next state for the procedure.
+   * @param stateId the ordinal() of the state enum (or state id)
+   */
+  private void setNextState(final int stateId) {
+    if (states == null || states.length == stateCount) {
+      int newCapacity = stateCount + 8;
+      if (states != null) {
+        states = Arrays.copyOf(states, newCapacity);
+      } else {
+        states = new int[newCapacity];
+      }
+    }
+    states[stateCount++] = stateId;
+  }
+
+  @Override
+  protected void serializeStateData(final OutputStream stream) throws IOException {
+    StateMachineProcedureData.Builder data = StateMachineProcedureData.newBuilder();
+    for (int i = 0; i < stateCount; ++i) {
+      data.addState(states[i]);
+    }
+    data.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  protected void deserializeStateData(final InputStream stream) throws IOException {
+    StateMachineProcedureData data = StateMachineProcedureData.parseDelimitedFrom(stream);
+    stateCount = data.getStateCount();
+    if (stateCount > 0) {
+      states = new int[stateCount];
+      for (int i = 0; i < stateCount; ++i) {
+        states[i] = data.getState(i);
+      }
+    } else {
+      states = null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TwoPhaseProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TwoPhaseProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TwoPhaseProcedure.java
new file mode 100644
index 0000000..cd6b0a7
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/TwoPhaseProcedure.java
@@ -0,0 +1,28 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class TwoPhaseProcedure<TEnvironment> extends Procedure<TEnvironment> {
+  // TODO (e.g. used by ACLs/VisibilityTags updates)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
new file mode 100644
index 0000000..0d1c050
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStore.java
@@ -0,0 +1,121 @@
+/**
+ * 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.procedure2.store;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+
+/**
+ * The ProcedureStore is used by the executor to persist the state of each procedure execution.
+ * This allows to resume the execution of pending/in-progress procedures in case
+ * of machine failure or service shutdown.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ProcedureStore {
+  /**
+   * Store listener interface.
+   * The main process should register a listener and respond to the store events.
+   */
+  public interface ProcedureStoreListener {
+    /**
+     * triggered when the store is not able to write out data.
+     * the main process should abort.
+     */
+    void abortProcess();
+  }
+
+  /**
+   * Add the listener to the notification list.
+   * @param listener The AssignmentListener to register
+   */
+  void registerListener(ProcedureStoreListener listener);
+
+  /**
+   * Remove the listener from the notification list.
+   * @param listener The AssignmentListener to unregister
+   * @return true if the listner was in the list and it was removed, otherwise false.
+   */
+  boolean unregisterListener(ProcedureStoreListener listener);
+
+  /**
+   * Start/Open the procedure store
+   * @param numThreads
+   */
+  void start(int numThreads) throws IOException;
+
+  /**
+   * Stop/Close the procedure store
+   * @param abort true if the stop is an abort
+   */
+  void stop(boolean abort);
+
+  /**
+   * @return true if the store is running, otherwise false.
+   */
+  boolean isRunning();
+
+  /**
+   * @return the number of threads/slots passed to start()
+   */
+  int getNumThreads();
+
+  /**
+   * Acquire the lease for the procedure store.
+   */
+  void recoverLease() throws IOException;
+
+  /**
+   * Load the Procedures in the store.
+   * @return the set of procedures present in the store
+   */
+  Iterator<Procedure> load() throws IOException;
+
+  /**
+   * When a procedure is submitted to the executor insert(proc, null) will be called.
+   * 'proc' has a 'RUNNABLE' state and the initial information required to start up.
+   *
+   * When a procedure is executed and it returns children insert(proc, subprocs) will be called.
+   * 'proc' has a 'WAITING' state and an update state.
+   * 'subprocs' are the children in 'RUNNABLE' state with the initial information.
+   *
+   * @param proc the procedure to serialize and write to the store.
+   * @param subprocs the newly created child of the proc.
+   */
+  void insert(Procedure proc, Procedure[] subprocs);
+
+  /**
+   * The specified procedure was executed,
+   * and the new state should be written to the store.
+   * @param proc the procedure to serialize and write to the store.
+   */
+  void update(Procedure proc);
+
+  /**
+   * The specified procId was removed from the executor,
+   * due to completion, abort or failure.
+   * The store implementor should remove all the information about the specified procId.
+   * @param procId the ID of the procedure to remove.
+   */
+  void delete(long procId);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
new file mode 100644
index 0000000..4e4653a
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
@@ -0,0 +1,540 @@
+/**
+ * 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.procedure2.store;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+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.protobuf.generated.ProcedureProtos;
+
+/**
+ * Keeps track of live procedures.
+ *
+ * It can be used by the ProcedureStore to identify which procedures are already
+ * deleted/completed to avoid the deserialization step on restart.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class ProcedureStoreTracker {
+  private final TreeMap<Long, BitSetNode> map = new TreeMap<Long, BitSetNode>();
+
+  private boolean keepDeletes = false;
+  private boolean partial = false;
+
+  public enum DeleteState { YES, NO, MAYBE }
+
+  public static class BitSetNode {
+    private final static long WORD_MASK = 0xffffffffffffffffL;
+    private final static int ADDRESS_BITS_PER_WORD = 6;
+    private final static int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD;
+    private final static int MAX_NODE_SIZE = 4 << ADDRESS_BITS_PER_WORD;
+
+    private long[] updated;
+    private long[] deleted;
+    private long start;
+
+    public void dump() {
+      System.out.printf("%06d:%06d min=%d max=%d%n", getStart(), getEnd(),
+        getMinProcId(), getMaxProcId());
+      System.out.println("Update:");
+      for (int i = 0; i < updated.length; ++i) {
+        for (int j = 0; j < BITS_PER_WORD; ++j) {
+          System.out.print((updated[i] & (1L << j)) != 0 ? "1" : "0");
+        }
+        System.out.println(" " + i);
+      }
+      System.out.println();
+      System.out.println("Delete:");
+      for (int i = 0; i < deleted.length; ++i) {
+        for (int j = 0; j < BITS_PER_WORD; ++j) {
+          System.out.print((deleted[i] & (1L << j)) != 0 ? "1" : "0");
+        }
+        System.out.println(" " + i);
+      }
+      System.out.println();
+    }
+
+    public BitSetNode(final long procId, final boolean partial) {
+      start = alignDown(procId);
+
+      int count = 2;
+      updated = new long[count];
+      deleted = new long[count];
+      for (int i = 0; i < count; ++i) {
+        updated[i] = 0;
+        deleted[i] = partial ? 0 : WORD_MASK;
+      }
+
+      updateState(procId, false);
+    }
+
+    protected BitSetNode(final long start, final long[] updated, final long[] deleted) {
+      this.start = start;
+      this.updated = updated;
+      this.deleted = deleted;
+    }
+
+    public void update(final long procId) {
+      updateState(procId, false);
+    }
+
+    public void delete(final long procId) {
+      updateState(procId, true);
+    }
+
+    public Long getStart() {
+      return start;
+    }
+
+    public Long getEnd() {
+      return start + (updated.length << ADDRESS_BITS_PER_WORD) - 1;
+    }
+
+    public boolean contains(final long procId) {
+      return start <= procId && procId <= getEnd();
+    }
+
+    public DeleteState isDeleted(final long procId) {
+      int bitmapIndex = getBitmapIndex(procId);
+      int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
+      if (wordIndex >= deleted.length) {
+        return DeleteState.MAYBE;
+      }
+      return (deleted[wordIndex] & (1L << bitmapIndex)) != 0 ? DeleteState.YES : DeleteState.NO;
+    }
+
+    private boolean isUpdated(final long procId) {
+      int bitmapIndex = getBitmapIndex(procId);
+      int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
+      if (wordIndex >= updated.length) {
+        return false;
+      }
+      return (updated[wordIndex] & (1L << bitmapIndex)) != 0;
+    }
+
+    public boolean isUpdated() {
+      // TODO: cache the value
+      for (int i = 0; i < updated.length; ++i) {
+        long deleteMask = ~deleted[i];
+        if ((updated[i] & deleteMask) != (WORD_MASK & deleteMask)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public boolean isEmpty() {
+      // TODO: cache the value
+      for (int i = 0; i < deleted.length; ++i) {
+        if (deleted[i] != WORD_MASK) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public void resetUpdates() {
+      for (int i = 0; i < updated.length; ++i) {
+        updated[i] = 0;
+      }
+    }
+
+    public void undeleteAll() {
+      for (int i = 0; i < updated.length; ++i) {
+        deleted[i] = 0;
+      }
+    }
+
+    public ProcedureProtos.ProcedureStoreTracker.TrackerNode convert() {
+      ProcedureProtos.ProcedureStoreTracker.TrackerNode.Builder builder =
+        ProcedureProtos.ProcedureStoreTracker.TrackerNode.newBuilder();
+      builder.setStartId(start);
+      for (int i = 0; i < updated.length; ++i) {
+        builder.addUpdated(updated[i]);
+        builder.addDeleted(deleted[i]);
+      }
+      return builder.build();
+    }
+
+    public static BitSetNode convert(ProcedureProtos.ProcedureStoreTracker.TrackerNode data) {
+      long start = data.getStartId();
+      int size = data.getUpdatedCount();
+      long[] updated = new long[size];
+      long[] deleted = new long[size];
+      for (int i = 0; i < size; ++i) {
+        updated[i] = data.getUpdated(i);
+        deleted[i] = data.getDeleted(i);
+      }
+      return new BitSetNode(start, updated, deleted);
+    }
+
+    // ========================================================================
+    //  Grow/Merge Helpers
+    // ========================================================================
+    public boolean canGrow(final long procId) {
+      return (procId - start) < MAX_NODE_SIZE;
+    }
+
+    public boolean canMerge(final BitSetNode rightNode) {
+      return (start + rightNode.getEnd()) < MAX_NODE_SIZE;
+    }
+
+    public void grow(final long procId) {
+      int delta, offset;
+
+      if (procId < start) {
+        // add to head
+        long newStart = alignDown(procId);
+        delta = (int)(start - newStart) >> ADDRESS_BITS_PER_WORD;
+        offset = delta;
+      } else {
+        // Add to tail
+        long newEnd = alignUp(procId + 1);
+        delta = (int)(newEnd - getEnd()) >> ADDRESS_BITS_PER_WORD;
+        offset = 0;
+      }
+
+      long[] newBitmap;
+      int oldSize = updated.length;
+
+      newBitmap = new long[oldSize + delta];
+      System.arraycopy(updated, 0, newBitmap, offset, oldSize);
+      updated = newBitmap;
+
+      newBitmap = new long[deleted.length + delta];
+      System.arraycopy(deleted, 0, newBitmap, offset, oldSize);
+      deleted = newBitmap;
+
+      for (int i = 0; i < delta; ++i) {
+        updated[oldSize + i] = 0;
+        deleted[oldSize + i] = WORD_MASK;
+      }
+    }
+
+    public void merge(final BitSetNode rightNode) {
+      int delta = (int)(rightNode.getEnd() - getEnd()) >> ADDRESS_BITS_PER_WORD;
+
+      long[] newBitmap;
+      int oldSize = updated.length;
+      int newSize = (delta - rightNode.updated.length);
+      int offset = oldSize + newSize;
+
+      newBitmap = new long[oldSize + delta];
+      System.arraycopy(updated, 0, newBitmap, 0, oldSize);
+      System.arraycopy(rightNode.updated, 0, newBitmap, offset, rightNode.updated.length);
+      updated = newBitmap;
+
+      newBitmap = new long[oldSize + delta];
+      System.arraycopy(deleted, 0, newBitmap, 0, oldSize);
+      System.arraycopy(rightNode.deleted, 0, newBitmap, offset, rightNode.deleted.length);
+      deleted = newBitmap;
+
+      for (int i = 0; i < newSize; ++i) {
+        updated[offset + i] = 0;
+        deleted[offset + i] = WORD_MASK;
+      }
+    }
+
+    // ========================================================================
+    //  Min/Max Helpers
+    // ========================================================================
+    public long getMinProcId() {
+      long minProcId = start;
+      for (int i = 0; i < deleted.length; ++i) {
+        if (deleted[i] == 0) {
+          return(minProcId);
+        }
+
+        if (deleted[i] != WORD_MASK) {
+          for (int j = 0; j < BITS_PER_WORD; ++j) {
+            if ((deleted[i] & (1L << j)) != 0) {
+              return minProcId + j;
+            }
+          }
+        }
+
+        minProcId += BITS_PER_WORD;
+      }
+      return minProcId;
+    }
+
+    public long getMaxProcId() {
+      long maxProcId = getEnd();
+      for (int i = deleted.length - 1; i >= 0; --i) {
+        if (deleted[i] == 0) {
+          return maxProcId;
+        }
+
+        if (deleted[i] != WORD_MASK) {
+          for (int j = BITS_PER_WORD - 1; j >= 0; --j) {
+            if ((deleted[i] & (1L << j)) == 0) {
+              return maxProcId - (BITS_PER_WORD - 1 - j);
+            }
+          }
+        }
+        maxProcId -= BITS_PER_WORD;
+      }
+      return maxProcId;
+    }
+
+    // ========================================================================
+    //  Bitmap Helpers
+    // ========================================================================
+    private int getBitmapIndex(final long procId) {
+      return (int)(procId - start);
+    }
+
+    private void updateState(final long procId, final boolean isDeleted) {
+      int bitmapIndex = getBitmapIndex(procId);
+      int wordIndex = bitmapIndex >> ADDRESS_BITS_PER_WORD;
+      long value = (1L << bitmapIndex);
+
+      if (isDeleted) {
+        updated[wordIndex] |= value;
+        deleted[wordIndex] |= value;
+      } else {
+        updated[wordIndex] |= value;
+        deleted[wordIndex] &= ~value;
+      }
+    }
+
+    // ========================================================================
+    //  Helpers
+    // ========================================================================
+    private static long alignUp(final long x) {
+      return (x + (BITS_PER_WORD - 1)) & -BITS_PER_WORD;
+    }
+
+    private static long alignDown(final long x) {
+      return x & -BITS_PER_WORD;
+    }
+  }
+
+  public void insert(final Procedure proc, final Procedure[] subprocs) {
+    insert(proc.getProcId());
+    if (subprocs != null) {
+      for (int i = 0; i < subprocs.length; ++i) {
+        insert(subprocs[i].getProcId());
+      }
+    }
+  }
+
+  public void update(final Procedure proc) {
+    update(proc.getProcId());
+  }
+
+  public void insert(long procId) {
+    BitSetNode node = getOrCreateNode(procId);
+    node.update(procId);
+  }
+
+  public void update(long procId) {
+    Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
+    assert entry != null : "expected node to update procId=" + procId;
+
+    BitSetNode node = entry.getValue();
+    assert node.contains(procId);
+    node.update(procId);
+  }
+
+  public void delete(long procId) {
+    Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
+    assert entry != null : "expected node to delete procId=" + procId;
+
+    BitSetNode node = entry.getValue();
+    assert node.contains(procId) : "expected procId in the node";
+    node.delete(procId);
+
+    if (!keepDeletes && node.isEmpty()) {
+      // TODO: RESET if (map.size() == 1)
+      map.remove(entry.getKey());
+    }
+  }
+
+  @InterfaceAudience.Private
+  public void setDeleted(final long procId, final boolean isDeleted) {
+    BitSetNode node = getOrCreateNode(procId);
+    node.updateState(procId, isDeleted);
+  }
+
+  public void clear() {
+    this.map.clear();
+  }
+
+  public DeleteState isDeleted(long procId) {
+    Map.Entry<Long, BitSetNode> entry = map.floorEntry(procId);
+    if (entry != null) {
+      BitSetNode node = entry.getValue();
+      DeleteState state = node.isDeleted(procId);
+      return partial && !node.isUpdated(procId) ? DeleteState.MAYBE : state;
+    }
+    return partial ? DeleteState.MAYBE : DeleteState.YES;
+  }
+
+  public long getMinProcId() {
+    // TODO: Cache?
+    Map.Entry<Long, BitSetNode> entry = map.firstEntry();
+    return entry == null ? 0 : entry.getValue().getMinProcId();
+  }
+
+  public void setKeepDeletes(boolean keepDeletes) {
+    this.keepDeletes = keepDeletes;
+    if (!keepDeletes) {
+      Iterator<Map.Entry<Long, BitSetNode>> it = map.entrySet().iterator();
+      while (it.hasNext()) {
+        Map.Entry<Long, BitSetNode> entry = it.next();
+        if (entry.getValue().isEmpty()) {
+          it.remove();
+        }
+      }
+    }
+  }
+
+  public void setPartialFlag(boolean isPartial) {
+    this.partial = isPartial;
+  }
+
+  public boolean isEmpty() {
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      if (entry.getValue().isEmpty() == false) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public boolean isUpdated() {
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      if (entry.getValue().isUpdated() == false) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public void resetUpdates() {
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      entry.getValue().resetUpdates();
+    }
+  }
+
+  public void undeleteAll() {
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      entry.getValue().undeleteAll();
+    }
+  }
+
+  private BitSetNode getOrCreateNode(final long procId) {
+    // can procId fit in the left node?
+    BitSetNode leftNode = null;
+    boolean leftCanGrow = false;
+    Map.Entry<Long, BitSetNode> leftEntry = map.floorEntry(procId);
+    if (leftEntry != null) {
+      leftNode = leftEntry.getValue();
+      if (leftNode.contains(procId)) {
+        return leftNode;
+      }
+      leftCanGrow = leftNode.canGrow(procId);
+    }
+
+    BitSetNode rightNode = null;
+    boolean rightCanGrow = false;
+    Map.Entry<Long, BitSetNode> rightEntry = map.ceilingEntry(procId);
+    if (rightEntry != null) {
+      rightNode = rightEntry.getValue();
+      rightCanGrow = rightNode.canGrow(procId);
+      if (leftNode != null) {
+        if (leftNode.canMerge(rightNode)) {
+          // merge left and right node
+          return mergeNodes(leftNode, rightNode);
+        }
+
+        if (leftCanGrow && rightCanGrow) {
+          if ((procId - leftNode.getEnd()) <= (rightNode.getStart() - procId)) {
+            // grow the left node
+            return growNode(leftNode, procId);
+          }
+          // grow the right node
+          return growNode(rightNode, procId);
+        }
+      }
+    }
+
+    // grow the left node
+    if (leftCanGrow) {
+      return growNode(leftNode, procId);
+    }
+
+    // grow the right node
+    if (rightCanGrow) {
+      return growNode(rightNode, procId);
+    }
+
+    // add new node
+    BitSetNode node = new BitSetNode(procId, partial);
+    map.put(node.getStart(), node);
+    return node;
+  }
+
+  private BitSetNode growNode(BitSetNode node, long procId) {
+    map.remove(node.getStart());
+    node.grow(procId);
+    map.put(node.getStart(), node);
+    return node;
+  }
+
+  private BitSetNode mergeNodes(BitSetNode leftNode, BitSetNode rightNode) {
+    leftNode.merge(rightNode);
+    map.remove(rightNode.getStart());
+    return leftNode;
+  }
+
+  public void dump() {
+    System.out.println("map " + map.size());
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      entry.getValue().dump();
+    }
+  }
+
+  public void writeTo(final OutputStream stream) throws IOException {
+    ProcedureProtos.ProcedureStoreTracker.Builder builder =
+        ProcedureProtos.ProcedureStoreTracker.newBuilder();
+    for (Map.Entry<Long, BitSetNode> entry : map.entrySet()) {
+      builder.addNode(entry.getValue().convert());
+    }
+    builder.build().writeDelimitedTo(stream);
+  }
+
+  public void readFrom(final InputStream stream) throws IOException {
+    ProcedureProtos.ProcedureStoreTracker data =
+        ProcedureProtos.ProcedureStoreTracker.parseDelimitedFrom(stream);
+    map.clear();
+    for (ProcedureProtos.ProcedureStoreTracker.TrackerNode protoNode: data.getNodeList()) {
+      BitSetNode node = BitSetNode.convert(protoNode);
+      map.put(node.getStart(), node);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/CorruptedWALProcedureStoreException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/CorruptedWALProcedureStoreException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/CorruptedWALProcedureStoreException.java
new file mode 100644
index 0000000..29db3bf
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/CorruptedWALProcedureStoreException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.procedure2.store.wal;
+
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Thrown when a procedure WAL is corrupted
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class CorruptedWALProcedureStoreException extends HBaseIOException {
+  /** default constructor */
+  public CorruptedWALProcedureStoreException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public CorruptedWALProcedureStoreException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
new file mode 100644
index 0000000..859b3cb
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -0,0 +1,152 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.IOException;
+
+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.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
+
+/**
+ * Describes a WAL File
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
+  private static final Log LOG = LogFactory.getLog(ProcedureWALFile.class);
+
+  private ProcedureWALHeader header;
+  private FSDataInputStream stream;
+  private FileStatus logStatus;
+  private FileSystem fs;
+  private Path logFile;
+  private long startPos;
+
+  public ProcedureWALFile(final FileSystem fs, final FileStatus logStatus) {
+    this.fs = fs;
+    this.logStatus = logStatus;
+    this.logFile = logStatus.getPath();
+  }
+
+  public ProcedureWALFile(FileSystem fs, Path logFile, ProcedureWALHeader header, long startPos) {
+    this.fs = fs;
+    this.logFile = logFile;
+    this.header = header;
+    this.startPos = startPos;
+  }
+
+  public void open() throws IOException {
+    if (stream == null) {
+      stream = fs.open(logFile);
+    }
+
+    if (header == null) {
+      header = ProcedureWALFormat.readHeader(stream);
+      startPos = stream.getPos();
+    } else {
+      stream.seek(startPos);
+    }
+  }
+
+  public ProcedureWALTrailer readTrailer() throws IOException {
+    try {
+      return ProcedureWALFormat.readTrailer(stream, startPos, logStatus.getLen());
+    } finally {
+      stream.seek(startPos);
+    }
+  }
+
+  public void readTracker(ProcedureStoreTracker tracker) throws IOException {
+    ProcedureWALTrailer trailer = readTrailer();
+    try {
+      stream.seek(trailer.getTrackerPos());
+      tracker.readFrom(stream);
+    } finally {
+      stream.seek(startPos);
+    }
+  }
+
+  public void close() {
+    if (stream == null) return;
+    try {
+      stream.close();
+    } catch (IOException e) {
+      LOG.warn("unable to close the wal file: " + logFile, e);
+    } finally {
+      stream = null;
+    }
+  }
+
+  public FSDataInputStream getStream() {
+    return stream;
+  }
+
+  public ProcedureWALHeader getHeader() {
+    return header;
+  }
+
+  public boolean isCompacted() {
+    return header.getType() == ProcedureWALFormat.LOG_TYPE_COMPACTED;
+  }
+
+  public long getLogId() {
+    return header.getLogId();
+  }
+
+  public long getSize() {
+    return logStatus.getLen();
+  }
+
+  public void removeFile() throws IOException {
+    close();
+    fs.delete(logFile, false);
+  }
+
+  @Override
+  public int compareTo(final ProcedureWALFile other) {
+    long diff = header.getLogId() - other.header.getLogId();
+    return (diff < 0) ? -1 : (diff > 0) ? 1 : 0;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof ProcedureWALFile)) return false;
+    return compareTo((ProcedureWALFile)o) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return logFile.hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return logFile.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
new file mode 100644
index 0000000..17432ac
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
@@ -0,0 +1,234 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.hbase.io.util.StreamUtils;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
+import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * Helper class that contains the WAL serialization utils.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class ProcedureWALFormat {
+  static final byte LOG_TYPE_STREAM = 0;
+  static final byte LOG_TYPE_COMPACTED = 1;
+  static final byte LOG_TYPE_MAX_VALID = 1;
+
+  static final byte HEADER_VERSION = 1;
+  static final byte TRAILER_VERSION = 1;
+  static final long HEADER_MAGIC = 0x31764c4157637250L;
+  static final long TRAILER_MAGIC = 0x50726357414c7631L;
+
+  @InterfaceAudience.Private
+  public static class InvalidWALDataException extends IOException {
+    public InvalidWALDataException(String s) {
+      super(s);
+    }
+
+    public InvalidWALDataException(Throwable t) {
+      super(t);
+    }
+  }
+
+  interface Loader {
+    void removeLog(ProcedureWALFile log);
+    void markCorruptedWAL(ProcedureWALFile log, IOException e);
+  }
+
+  private ProcedureWALFormat() {}
+
+  public static Iterator<Procedure> load(final Iterator<ProcedureWALFile> logs,
+      final ProcedureStoreTracker tracker, final Loader loader) throws IOException {
+    ProcedureWALFormatReader reader = new ProcedureWALFormatReader(tracker);
+    tracker.setKeepDeletes(true);
+    try {
+      while (logs.hasNext()) {
+        ProcedureWALFile log = logs.next();
+        log.open();
+        try {
+          reader.read(log, loader);
+        } finally {
+          log.close();
+        }
+      }
+      // The tracker is now updated with all the procedures read from the logs
+      tracker.setPartialFlag(false);
+      tracker.resetUpdates();
+    } finally {
+      tracker.setKeepDeletes(false);
+    }
+    // TODO: Write compacted version?
+    return reader.getProcedures();
+  }
+
+  public static void writeHeader(OutputStream stream, ProcedureWALHeader header)
+      throws IOException {
+    header.writeDelimitedTo(stream);
+  }
+
+  /*
+   * +-----------------+
+   * | END OF WAL DATA | <---+
+   * +-----------------+     |
+   * |                 |     |
+   * |     Tracker     |     |
+   * |                 |     |
+   * +-----------------+     |
+   * |     version     |     |
+   * +-----------------+     |
+   * |  TRAILER_MAGIC  |     |
+   * +-----------------+     |
+   * |      offset     |-----+
+   * +-----------------+
+   */
+  public static void writeTrailer(FSDataOutputStream stream, ProcedureStoreTracker tracker)
+      throws IOException {
+    long offset = stream.getPos();
+
+    // Write EOF Entry
+    ProcedureWALEntry.newBuilder()
+      .setType(ProcedureWALEntry.Type.EOF)
+      .build().writeDelimitedTo(stream);
+
+    // Write Tracker
+    tracker.writeTo(stream);
+
+    stream.write(TRAILER_VERSION);
+    StreamUtils.writeLong(stream, TRAILER_MAGIC);
+    StreamUtils.writeLong(stream, offset);
+  }
+
+  public static ProcedureWALHeader readHeader(InputStream stream)
+      throws IOException {
+    ProcedureWALHeader header;
+    try {
+      header = ProcedureWALHeader.parseDelimitedFrom(stream);
+    } catch (InvalidProtocolBufferException e) {
+      throw new InvalidWALDataException(e);
+    }
+
+    if (header == null) {
+      throw new InvalidWALDataException("No data available to read the Header");
+    }
+
+    if (header.getVersion() < 0 || header.getVersion() != HEADER_VERSION) {
+      throw new InvalidWALDataException("Invalid Header version. got " + header.getVersion() +
+          " expected " + HEADER_VERSION);
+    }
+
+    if (header.getType() < 0 || header.getType() > LOG_TYPE_MAX_VALID) {
+      throw new InvalidWALDataException("Invalid header type. got " + header.getType());
+    }
+
+    return header;
+  }
+
+  public static ProcedureWALTrailer readTrailer(FSDataInputStream stream, long startPos, long size)
+      throws IOException {
+    long trailerPos = size - 17; // Beginning of the Trailer Jump
+
+    if (trailerPos < startPos) {
+      throw new InvalidWALDataException("Missing trailer: size=" + size + " startPos=" + startPos);
+    }
+
+    stream.seek(trailerPos);
+    int version = stream.read();
+    if (version != TRAILER_VERSION) {
+      throw new InvalidWALDataException("Invalid Trailer version. got " + version +
+          " expected " + TRAILER_VERSION);
+    }
+
+    long magic = StreamUtils.readLong(stream);
+    if (magic != TRAILER_MAGIC) {
+      throw new InvalidWALDataException("Invalid Trailer magic. got " + magic +
+          " expected " + TRAILER_MAGIC);
+    }
+
+    long trailerOffset = StreamUtils.readLong(stream);
+    stream.seek(trailerOffset);
+
+    ProcedureWALEntry entry = readEntry(stream);
+    if (entry.getType() != ProcedureWALEntry.Type.EOF) {
+      throw new InvalidWALDataException("Invalid Trailer begin");
+    }
+
+    ProcedureWALTrailer trailer = ProcedureWALTrailer.newBuilder()
+      .setVersion(version)
+      .setTrackerPos(stream.getPos())
+      .build();
+    return trailer;
+  }
+
+  public static ProcedureWALEntry readEntry(InputStream stream) throws IOException {
+    return ProcedureWALEntry.parseDelimitedFrom(stream);
+  }
+
+  public static void writeEntry(ByteSlot slot, ProcedureWALEntry.Type type,
+      Procedure proc, Procedure[] subprocs) throws IOException {
+    ProcedureWALEntry.Builder builder = ProcedureWALEntry.newBuilder();
+    builder.setType(type);
+    builder.addProcedure(Procedure.convert(proc));
+    if (subprocs != null) {
+      for (int i = 0; i < subprocs.length; ++i) {
+        builder.addProcedure(Procedure.convert(subprocs[i]));
+      }
+    }
+    builder.build().writeDelimitedTo(slot);
+  }
+
+  public static void writeInsert(ByteSlot slot, Procedure proc)
+      throws IOException {
+    writeEntry(slot, ProcedureWALEntry.Type.INIT, proc, null);
+  }
+
+  public static void writeInsert(ByteSlot slot, Procedure proc, Procedure[] subprocs)
+      throws IOException {
+    writeEntry(slot, ProcedureWALEntry.Type.INSERT, proc, subprocs);
+  }
+
+  public static void writeUpdate(ByteSlot slot, Procedure proc)
+      throws IOException {
+    writeEntry(slot, ProcedureWALEntry.Type.UPDATE, proc, null);
+  }
+
+  public static void writeDelete(ByteSlot slot, long procId)
+      throws IOException {
+    ProcedureWALEntry.Builder builder = ProcedureWALEntry.newBuilder();
+    builder.setType(ProcedureWALEntry.Type.DELETE);
+    builder.setProcId(procId);
+    builder.build().writeDelimitedTo(slot);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
new file mode 100644
index 0000000..a60b8f5
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
@@ -0,0 +1,166 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.HashMap;
+
+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.fs.FSDataInputStream;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
+
+/**
+ * Helper class that loads the procedures stored in a WAL
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureWALFormatReader {
+  private static final Log LOG = LogFactory.getLog(ProcedureWALFormatReader.class);
+
+  private final ProcedureStoreTracker tracker;
+  //private final long compactionLogId;
+
+  private final Map<Long, Procedure> procedures = new HashMap<Long, Procedure>();
+  private final Map<Long, ProcedureProtos.Procedure> localProcedures =
+    new HashMap<Long, ProcedureProtos.Procedure>();
+
+  private long maxProcId = 0;
+
+  public ProcedureWALFormatReader(final ProcedureStoreTracker tracker) {
+    this.tracker = tracker;
+  }
+
+  public void read(ProcedureWALFile log, ProcedureWALFormat.Loader loader) throws IOException {
+    FSDataInputStream stream = log.getStream();
+    try {
+      boolean hasMore = true;
+      while (hasMore) {
+        ProcedureWALEntry entry = ProcedureWALFormat.readEntry(stream);
+        if (entry == null) {
+          LOG.warn("nothing left to decode. exiting with missing EOF");
+          hasMore = false;
+          break;
+        }
+        switch (entry.getType()) {
+          case INIT:
+            readInitEntry(entry);
+            break;
+          case INSERT:
+            readInsertEntry(entry);
+            break;
+          case UPDATE:
+          case COMPACT:
+            readUpdateEntry(entry);
+            break;
+          case DELETE:
+            readDeleteEntry(entry);
+            break;
+          case EOF:
+            hasMore = false;
+            break;
+          default:
+            throw new CorruptedWALProcedureStoreException("Invalid entry: " + entry);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("got an exception while reading the procedure WAL: " + log, e);
+      loader.markCorruptedWAL(log, e);
+    }
+
+    if (localProcedures.isEmpty()) {
+      LOG.info("No active entry found in state log " + log + ". removing it");
+      loader.removeLog(log);
+    } else {
+      Iterator<Map.Entry<Long, ProcedureProtos.Procedure>> itd =
+        localProcedures.entrySet().iterator();
+      while (itd.hasNext()) {
+        Map.Entry<Long, ProcedureProtos.Procedure> entry = itd.next();
+        itd.remove();
+
+        // Deserialize the procedure
+        Procedure proc = Procedure.convert(entry.getValue());
+        procedures.put(entry.getKey(), proc);
+      }
+
+      // TODO: Some procedure may be already runnables (see readInitEntry())
+      //       (we can also check the "update map" in the log trackers)
+    }
+  }
+
+  public Iterator<Procedure> getProcedures() {
+    return procedures.values().iterator();
+  }
+
+  private void loadEntries(final ProcedureWALEntry entry) {
+    for (ProcedureProtos.Procedure proc: entry.getProcedureList()) {
+      maxProcId = Math.max(maxProcId, proc.getProcId());
+      if (isRequired(proc.getProcId())) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("read " + entry.getType() + " entry " + proc.getProcId());
+        }
+        localProcedures.put(proc.getProcId(), proc);
+        tracker.setDeleted(proc.getProcId(), false);
+      }
+    }
+  }
+
+  private void readInitEntry(final ProcedureWALEntry entry)
+      throws IOException {
+    assert entry.getProcedureCount() == 1 : "Expected only one procedure";
+    // TODO: Make it runnable, before reading other files
+    loadEntries(entry);
+  }
+
+  private void readInsertEntry(final ProcedureWALEntry entry) throws IOException {
+    assert entry.getProcedureCount() >= 1 : "Expected one or more procedures";
+    loadEntries(entry);
+  }
+
+  private void readUpdateEntry(final ProcedureWALEntry entry) throws IOException {
+    assert entry.getProcedureCount() == 1 : "Expected only one procedure";
+    loadEntries(entry);
+  }
+
+  private void readDeleteEntry(final ProcedureWALEntry entry) throws IOException {
+    assert entry.getProcedureCount() == 0 : "Expected no procedures";
+    assert entry.hasProcId() : "expected ProcID";
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("read delete entry " + entry.getProcId());
+    }
+    maxProcId = Math.max(maxProcId, entry.getProcId());
+    localProcedures.remove(entry.getProcId());
+    tracker.setDeleted(entry.getProcId(), true);
+  }
+
+  private boolean isDeleted(final long procId) {
+    return tracker.isDeleted(procId) == ProcedureStoreTracker.DeleteState.YES;
+  }
+
+  private boolean isRequired(final long procId) {
+    return !isDeleted(procId) && !procedures.containsKey(procId);
+  }
+}
\ No newline at end of file


[03/28] hbase git commit: HBASE-13118 [PE] Add being able to write many columns

Posted by mb...@apache.org.
HBASE-13118 [PE] Add being able to write many columns

Conflicts:
	hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java


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

Branch: refs/heads/hbase-12439
Commit: 66f7bf461532423d92be4e57a26edc2e13a7dbad
Parents: ed70376
Author: stack <st...@apache.org>
Authored: Thu Apr 9 16:49:16 2015 -0700
Committer: stack <st...@apache.org>
Committed: Thu Apr 9 16:52:11 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/PerformanceEvaluation.java     | 152 +++++++++++++------
 .../hbase/mapreduce/TestHFileOutputFormat.java  |   9 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |   9 +-
 3 files changed, 115 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/66f7bf46/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index d51f698..eb2f468 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -124,7 +124,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
   public static final String TABLE_NAME = "TestTable";
   public static final byte[] FAMILY_NAME = Bytes.toBytes("info");
-  public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
+  public static final byte [] COLUMN_ZERO = Bytes.toBytes("" + 0);
+  public static final byte [] QUALIFIER_NAME = COLUMN_ZERO;
   public static final int DEFAULT_VALUE_LENGTH = 1000;
   public static final int ROW_LENGTH = 26;
 
@@ -610,6 +611,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     int valueSize = DEFAULT_VALUE_LENGTH;
     int period = (this.perClientRunRows / 10) == 0? perClientRunRows: perClientRunRows / 10;
     int cycles = 1;
+    int columns = 1;
+    int caching = 30;
     boolean addColumns = true;
 
     public TestOptions() {}
@@ -653,6 +656,24 @@ public class PerformanceEvaluation extends Configured implements Tool {
       this.randomSleep = that.randomSleep;
       this.measureAfter = that.measureAfter;
       this.addColumns = that.addColumns;
+      this.columns = that.columns;
+      this.caching = that.caching;
+    }
+
+    public int getCaching() {
+      return this.caching;
+    }
+
+    public void setCaching(final int caching) {
+      this.caching = caching;
+    }
+
+    public int getColumns() {
+      return this.columns;
+    }
+
+    public void setColumns(final int columns) {
+      this.columns = columns;
     }
 
     public int getCycles() {
@@ -1157,7 +1178,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void onStartup() throws IOException {
       this.table = connection.getTable(TableName.valueOf(opts.tableName));
     }
-    
+
     @Override
     void onTakedown() throws IOException {
       table.close();
@@ -1175,7 +1196,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void onStartup() throws IOException {
       this.mutator = connection.getBufferedMutator(TableName.valueOf(opts.tableName));
     }
-    
+
     @Override
     void onTakedown() throws IOException {
       mutator.close();
@@ -1190,9 +1211,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
     @Override
     void testRow(final int i) throws IOException {
       Scan scan = new Scan(getRandomRow(this.rand, opts.totalRows));
+      scan.setCaching(opts.caching);
       FilterList list = new FilterList();
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      } else {
+        scan.addFamily(FAMILY_NAME);
       }
       if (opts.filterAll) {
         list.addFilter(new FilterAllFilter());
@@ -1223,11 +1247,14 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       Pair<byte[], byte[]> startAndStopRow = getStartAndStopRow();
       Scan scan = new Scan(startAndStopRow.getFirst(), startAndStopRow.getSecond());
+      scan.setCaching(opts.caching);
       if (opts.filterAll) {
         scan.setFilter(new FilterAllFilter());
       }
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      } else {
+        scan.addFamily(FAMILY_NAME);
       }
       Result r = null;
       int count = 0;
@@ -1326,6 +1353,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
       Get get = new Get(getRandomRow(this.rand, opts.totalRows));
       if (opts.addColumns) {
         get.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      } else {
+        get.addFamily(FAMILY_NAME);
       }
       if (opts.filterAll) {
         get.setFilter(new FilterAllFilter());
@@ -1369,21 +1398,24 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       byte[] row = getRandomRow(this.rand, opts.totalRows);
       Put put = new Put(row);
-      byte[] value = generateData(this.rand, getValueLength(this.rand));
-      if (opts.useTags) {
-        byte[] tag = generateData(this.rand, TAG_LENGTH);
-        Tag[] tags = new Tag[opts.noOfTags];
-        for (int n = 0; n < opts.noOfTags; n++) {
-          Tag t = new Tag((byte) n, tag);
-          tags[n] = t;
+      for (int column = 0; column < opts.columns; column++) {
+        byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+        byte[] value = generateData(this.rand, getValueLength(this.rand));
+        if (opts.useTags) {
+          byte[] tag = generateData(this.rand, TAG_LENGTH);
+          Tag[] tags = new Tag[opts.noOfTags];
+          for (int n = 0; n < opts.noOfTags; n++) {
+            Tag t = new Tag((byte) n, tag);
+            tags[n] = t;
+          }
+          KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
+              value, tags);
+          put.add(kv);
+          updateValueSize(kv.getValueLength());
+        } else {
+          put.add(FAMILY_NAME, qualifier, value);
+          updateValueSize(value.length);
         }
-        KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP,
-            value, tags);
-        put.add(kv);
-        updateValueSize(kv.getValueLength());
-      } else {
-        put.add(FAMILY_NAME, QUALIFIER_NAME, value);
-        updateValueSize(value.length);
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
       mutator.mutate(put);
@@ -1410,9 +1442,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       if (this.testScanner == null) {
         Scan scan = new Scan(format(opts.startRow));
-        scan.setCaching(30);
+        scan.setCaching(opts.caching);
         if (opts.addColumns) {
           scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+        } else {
+          scan.addFamily(FAMILY_NAME);
         }
         if (opts.filterAll) {
           scan.setFilter(new FilterAllFilter());
@@ -1452,21 +1486,24 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testRow(final int i) throws IOException {
       byte[] row = format(i);
       Put put = new Put(row);
-      byte[] value = generateData(this.rand, getValueLength(this.rand));
-      if (opts.useTags) {
-        byte[] tag = generateData(this.rand, TAG_LENGTH);
-        Tag[] tags = new Tag[opts.noOfTags];
-        for (int n = 0; n < opts.noOfTags; n++) {
-          Tag t = new Tag((byte) n, tag);
-          tags[n] = t;
+      for (int column = 0; column < opts.columns; column++) {
+        byte [] qualifier = column == 0? COLUMN_ZERO: Bytes.toBytes("" + column);
+        byte[] value = generateData(this.rand, getValueLength(this.rand));
+        if (opts.useTags) {
+          byte[] tag = generateData(this.rand, TAG_LENGTH);
+          Tag[] tags = new Tag[opts.noOfTags];
+          for (int n = 0; n < opts.noOfTags; n++) {
+            Tag t = new Tag((byte) n, tag);
+            tags[n] = t;
+          }
+          KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
+              value, tags);
+          put.add(kv);
+          updateValueSize(kv.getValueLength());
+        } else {
+          put.add(FAMILY_NAME, qualifier, value);
+          updateValueSize(value.length);
         }
-        KeyValue kv = new KeyValue(row, FAMILY_NAME, QUALIFIER_NAME, HConstants.LATEST_TIMESTAMP,
-            value, tags);
-        put.add(kv);
-        updateValueSize(kv.getValueLength());
-      } else {
-        put.add(FAMILY_NAME, QUALIFIER_NAME, value);
-        updateValueSize(value.length);
       }
       put.setDurability(opts.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
       mutator.mutate(put);
@@ -1498,7 +1535,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     protected Scan constructScan(byte[] valuePrefix) throws IOException {
       FilterList list = new FilterList();
       Filter filter = new SingleColumnValueFilter(
-          FAMILY_NAME, QUALIFIER_NAME, CompareFilter.CompareOp.EQUAL,
+          FAMILY_NAME, COLUMN_ZERO, CompareFilter.CompareOp.EQUAL,
           new BinaryComparator(valuePrefix)
       );
       list.addFilter(filter);
@@ -1506,8 +1543,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
         list.addFilter(new FilterAllFilter());
       }
       Scan scan = new Scan();
+      scan.setCaching(opts.caching);
       if (opts.addColumns) {
         scan.addColumn(FAMILY_NAME, QUALIFIER_NAME);
+      } else {
+        scan.addFamily(FAMILY_NAME);
       }
       scan.setFilter(list);
       return scan;
@@ -1520,11 +1560,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * @param timeMs Time taken in milliseconds.
    * @return String value with label, ie '123.76 MB/s'
    */
-  private static String calculateMbps(int rows, long timeMs, final int valueSize) {
-    // MB/s = ((totalRows * ROW_SIZE_BYTES) / totalTimeMS)
-    //        * 1000 MS_PER_SEC / (1024 * 1024) BYTES_PER_MB
-    BigDecimal rowSize =
-      BigDecimal.valueOf(ROW_LENGTH + valueSize + FAMILY_NAME.length + QUALIFIER_NAME.length);
+  private static String calculateMbps(int rows, long timeMs, final int valueSize, int columns) {
+    BigDecimal rowSize = BigDecimal.valueOf(ROW_LENGTH +
+      ((valueSize + FAMILY_NAME.length + COLUMN_ZERO.length) * columns));
     BigDecimal mbps = BigDecimal.valueOf(rows).multiply(rowSize, CXT)
       .divide(BigDecimal.valueOf(timeMs), CXT).multiply(MS_PER_SEC, CXT)
       .divide(BYTES_PER_MB, CXT);
@@ -1613,7 +1651,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     status.setStatus("Finished " + cmd + " in " + totalElapsedTime +
       "ms at offset " + opts.startRow + " for " + opts.perClientRunRows + " rows" +
       " (" + calculateMbps((int)(opts.perClientRunRows * opts.sampleRate), totalElapsedTime,
-          getAverageValueLength(opts)) + ")");
+          getAverageValueLength(opts), opts.columns) + ")");
 
     return new RunResult(totalElapsedTime, t.getLatency());
   }
@@ -1645,14 +1683,23 @@ public class PerformanceEvaluation extends Configured implements Tool {
   }
 
   protected void printUsage() {
-    printUsage(null);
+    printUsage(this.getClass().getName(), null);
+  }
+
+  protected static void printUsage(final String message) {
+    printUsage(PerformanceEvaluation.class.getName(), message);
+  }
+
+  protected static void printUsageAndExit(final String message, final int exitCode) {
+    printUsage(message);
+    System.exit(exitCode);
   }
 
-  protected void printUsage(final String message) {
+  protected static void printUsage(final String className, final String message) {
     if (message != null && message.length() > 0) {
       System.err.println(message);
     }
-    System.err.println("Usage: java " + this.getClass().getName() + " \\");
+    System.err.println("Usage: java " + className + " \\");
     System.err.println("  <OPTIONS> [-D<property=value>]* <command> <nclients>");
     System.err.println();
     System.err.println("Options:");
@@ -1704,6 +1751,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println(" cycles          How many times to cycle the test. Defaults: 1.");
     System.err.println(" splitPolicy     Specify a custom RegionSplitPolicy for the table.");
     System.err.println(" randomSleep     Do a random sleep before each get between 0 and entered value. Defaults: 0");
+    System.err.println(" columns         Columns to write per row. Default: 1");
+    System.err.println(" caching         Scan caching to use. Default: 30");
     System.err.println();
     System.err.println(" Note: -D properties will be applied to the conf used. ");
     System.err.println("  For example: ");
@@ -1721,8 +1770,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     System.err.println("                 running: 1 <= value <= 500");
     System.err.println("Examples:");
     System.err.println(" To run a single evaluation client:");
-    System.err.println(" $ bin/hbase " + this.getClass().getName()
-        + " sequentialWrite 1");
+    System.err.println(" $ bin/hbase " + className + " sequentialWrite 1");
   }
 
   /**
@@ -1935,6 +1983,18 @@ public class PerformanceEvaluation extends Configured implements Tool {
         continue;
       }
 
+      final String columns = "--columns=";
+      if (cmd.startsWith(columns)) {
+        opts.columns = Integer.parseInt(cmd.substring(columns.length()));
+        continue;
+      }
+
+      final String caching = "--caching=";
+      if (cmd.startsWith(caching)) {
+        opts.caching = Integer.parseInt(cmd.substring(caching.length()));
+        continue;
+      }
+
       if (isCommandClass(cmd)) {
         opts.cmdName = cmd;
         opts.numClientThreads = Integer.parseInt(args.remove());
@@ -1945,6 +2005,8 @@ public class PerformanceEvaluation extends Configured implements Tool {
         }
         opts = calculateRowsAndSize(opts);
         break;
+      } else {
+        printUsageAndExit("ERROR: Unrecognized option/command: " + cmd, -1);
       }
 
       // Not matching any option or command.
@@ -1970,7 +2032,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
   }
 
   static int getRowsPerGB(final TestOptions opts) {
-    return ONE_GB / (opts.valueRandom? opts.valueSize/2: opts.valueSize);
+    return ONE_GB / ((opts.valueRandom? opts.valueSize/2: opts.valueSize) * opts.getColumns());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f7bf46/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
index ecea98e..438266e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
@@ -125,6 +125,7 @@ public class TestHFileOutputFormat  {
     private int valLength;
     private static final int VALLEN_DEFAULT=10;
     private static final String VALLEN_CONF="randomkv.val.length";
+    private static final byte [] QUALIFIER = Bytes.toBytes("data");
 
     @Override
     protected void setup(Context context) throws IOException,
@@ -159,8 +160,7 @@ public class TestHFileOutputFormat  {
         ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
 
         for (byte[] family : TestHFileOutputFormat.FAMILIES) {
-          KeyValue kv = new KeyValue(keyBytes, family,
-              PerformanceEvaluation.QUALIFIER_NAME, valBytes);
+          KeyValue kv = new KeyValue(keyBytes, family, QUALIFIER, valBytes);
           context.write(key, kv);
         }
       }
@@ -878,7 +878,7 @@ public class TestHFileOutputFormat  {
 
     int taskId = context.getTaskAttemptID().getTaskID().getId();
     assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
-
+    final byte [] qualifier = Bytes.toBytes("data");
     Random random = new Random();
     for (int i = 0; i < numRows; i++) {
 
@@ -887,8 +887,7 @@ public class TestHFileOutputFormat  {
       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
 
       for (byte[] family : families) {
-        KeyValue kv = new KeyValue(keyBytes, family,
-            PerformanceEvaluation.QUALIFIER_NAME, valBytes);
+        KeyValue kv = new KeyValue(keyBytes, family, qualifier, valBytes);
         writer.write(key, kv);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/66f7bf46/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index 0f60f3b..67a6c0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -124,6 +124,7 @@ public class TestHFileOutputFormat2  {
     private int valLength;
     private static final int VALLEN_DEFAULT=10;
     private static final String VALLEN_CONF="randomkv.val.length";
+    private static final byte [] QUALIFIER = Bytes.toBytes("data");
 
     @Override
     protected void setup(Context context) throws IOException,
@@ -159,8 +160,7 @@ public class TestHFileOutputFormat2  {
         ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
 
         for (byte[] family : TestHFileOutputFormat2.FAMILIES) {
-          Cell kv = new KeyValue(keyBytes, family,
-              PerformanceEvaluation.QUALIFIER_NAME, valBytes);
+          Cell kv = new KeyValue(keyBytes, family, QUALIFIER, valBytes);
           context.write(key, kv);
         }
       }
@@ -879,7 +879,7 @@ public class TestHFileOutputFormat2  {
 
     int taskId = context.getTaskAttemptID().getTaskID().getId();
     assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
-
+    final byte [] qualifier = Bytes.toBytes("data");
     Random random = new Random();
     for (int i = 0; i < numRows; i++) {
 
@@ -888,8 +888,7 @@ public class TestHFileOutputFormat2  {
       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
 
       for (byte[] family : families) {
-        Cell kv = new KeyValue(keyBytes, family,
-            PerformanceEvaluation.QUALIFIER_NAME, valBytes);
+        Cell kv = new KeyValue(keyBytes, family, qualifier, valBytes);
         writer.write(key, kv);
       }
     }


[11/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
new file mode 100644
index 0000000..0669549
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/TestProcedureStoreTracker.java
@@ -0,0 +1,168 @@
+/**
+ * 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.procedure2.store;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.Assert;
+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 TestProcedureStoreTracker {
+  private static final Log LOG = LogFactory.getLog(TestProcedureStoreTracker.class);
+
+  static class TestProcedure extends Procedure<Void> {
+    public TestProcedure(long procId) {
+      setProcId(procId);
+    }
+
+    @Override
+    protected Procedure[] execute(Void env) { return null; }
+
+    @Override
+    protected void rollback(Void env) { /* no-op */ }
+
+    @Override
+    protected boolean abort(Void env) { return false; }
+
+    @Override
+    protected void serializeStateData(final OutputStream stream) { /* no-op */ }
+
+    @Override
+    protected void deserializeStateData(final InputStream stream) { /* no-op */ }
+  }
+
+  @Test
+  public void testSeqInsertAndDelete() {
+    ProcedureStoreTracker tracker = new ProcedureStoreTracker();
+    assertTrue(tracker.isEmpty());
+
+    final int MIN_PROC = 1;
+    final int MAX_PROC = 1 << 10;
+
+    // sequential insert
+    for (int i = MIN_PROC; i < MAX_PROC; ++i) {
+      tracker.insert(i);
+
+      // All the proc that we inserted should not be deleted
+      for (int j = MIN_PROC; j <= i; ++j) {
+        assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(j));
+      }
+      // All the proc that are not yet inserted should be result as deleted
+      for (int j = i + 1; j < MAX_PROC; ++j) {
+        assertTrue(tracker.isDeleted(j) != ProcedureStoreTracker.DeleteState.NO);
+      }
+    }
+
+    // sequential delete
+    for (int i = MIN_PROC; i < MAX_PROC; ++i) {
+      tracker.delete(i);
+
+      // All the proc that we deleted should be deleted
+      for (int j = MIN_PROC; j <= i; ++j) {
+        assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(j));
+      }
+      // All the proc that are not yet deleted should be result as not deleted
+      for (int j = i + 1; j < MAX_PROC; ++j) {
+        assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(j));
+      }
+    }
+    assertTrue(tracker.isEmpty());
+  }
+
+  @Test
+  public void testPartialTracker() {
+    ProcedureStoreTracker tracker = new ProcedureStoreTracker();
+    tracker.setPartialFlag(true);
+
+    // nothing in the tracker, the state is unknown
+    assertTrue(tracker.isEmpty());
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(1));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(579));
+
+    // Mark 1 as deleted, now that is a known state
+    tracker.setDeleted(1, true);
+    tracker.dump();
+    assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(1));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(2));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(579));
+
+    // Mark 579 as non-deleted, now that is a known state
+    tracker.setDeleted(579, false);
+    assertEquals(ProcedureStoreTracker.DeleteState.YES, tracker.isDeleted(1));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(2));
+    assertEquals(ProcedureStoreTracker.DeleteState.NO, tracker.isDeleted(579));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(577));
+    assertEquals(ProcedureStoreTracker.DeleteState.MAYBE, tracker.isDeleted(580));
+  }
+
+  @Test
+  public void testBasicCRUD() {
+    ProcedureStoreTracker tracker = new ProcedureStoreTracker();
+    assertTrue(tracker.isEmpty());
+
+    Procedure[] procs = new TestProcedure[] {
+      new TestProcedure(1), new TestProcedure(2), new TestProcedure(3),
+      new TestProcedure(4), new TestProcedure(5), new TestProcedure(6),
+    };
+
+    tracker.insert(procs[0], null);
+    tracker.insert(procs[1], new Procedure[] { procs[2], procs[3], procs[4] });
+    assertFalse(tracker.isEmpty());
+    assertTrue(tracker.isUpdated());
+
+    tracker.resetUpdates();
+    assertFalse(tracker.isUpdated());
+
+    for (int i = 0; i < 4; ++i) {
+      tracker.update(procs[i]);
+      assertFalse(tracker.isEmpty());
+      assertFalse(tracker.isUpdated());
+    }
+
+    tracker.update(procs[4]);
+    assertFalse(tracker.isEmpty());
+    assertTrue(tracker.isUpdated());
+
+    tracker.update(procs[5]);
+    assertFalse(tracker.isEmpty());
+    assertTrue(tracker.isUpdated());
+
+    for (int i = 0; i < 5; ++i) {
+      tracker.delete(procs[i].getProcId());
+      assertFalse(tracker.isEmpty());
+      assertTrue(tracker.isUpdated());
+    }
+    tracker.delete(procs[5].getProcId());
+    assertTrue(tracker.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
new file mode 100644
index 0000000..344b28b
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/store/wal/TestWALProcedureStore.java
@@ -0,0 +1,267 @@
+/**
+ * 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.procedure2.store.wal;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Iterator;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.SequentialProcedure;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.IOUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Assert;
+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 TestWALProcedureStore {
+  private static final Log LOG = LogFactory.getLog(TestWALProcedureStore.class);
+
+  private static final int PROCEDURE_STORE_SLOTS = 1;
+  private static final Procedure NULL_PROC = null;
+
+  private WALProcedureStore procStore;
+
+  private HBaseCommonTestingUtility htu;
+  private FileSystem fs;
+  private Path testDir;
+  private Path logDir;
+
+  @Before
+  public void setUp() throws IOException {
+    htu = new HBaseCommonTestingUtility();
+    testDir = htu.getDataTestDir();
+    fs = testDir.getFileSystem(htu.getConfiguration());
+    assertTrue(testDir.depth() > 1);
+
+    logDir = new Path(testDir, "proc-logs");
+    procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), fs, logDir);
+    procStore.start(PROCEDURE_STORE_SLOTS);
+    procStore.recoverLease();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    procStore.stop(false);
+    fs.delete(logDir, true);
+  }
+
+  private Iterator<Procedure> storeRestart() throws Exception {
+    procStore.stop(false);
+    procStore.start(PROCEDURE_STORE_SLOTS);
+    procStore.recoverLease();
+    return procStore.load();
+  }
+
+  @Test
+  public void testEmptyLogLoad() throws Exception {
+    Iterator<Procedure> loader = storeRestart();
+    assertEquals(0, countProcedures(loader));
+  }
+
+  @Test
+  public void testLoad() throws Exception {
+    Set<Long> procIds = new HashSet<>();
+
+    // Insert something in the log
+    Procedure proc1 = new TestSequentialProcedure();
+    procIds.add(proc1.getProcId());
+    procStore.insert(proc1, null);
+
+    Procedure proc2 = new TestSequentialProcedure();
+    Procedure[] child2 = new Procedure[2];
+    child2[0] = new TestSequentialProcedure();
+    child2[1] = new TestSequentialProcedure();
+
+    procIds.add(proc2.getProcId());
+    procIds.add(child2[0].getProcId());
+    procIds.add(child2[1].getProcId());
+    procStore.insert(proc2, child2);
+
+    // Verify that everything is there
+    verifyProcIdsOnRestart(procIds);
+
+    // Update and delete something
+    procStore.update(proc1);
+    procStore.update(child2[1]);
+    procStore.delete(child2[1].getProcId());
+    procIds.remove(child2[1].getProcId());
+
+    // Verify that everything is there
+    verifyProcIdsOnRestart(procIds);
+
+    // Remove 4 byte from the trailers
+    procStore.stop(false);
+    FileStatus[] logs = fs.listStatus(logDir);
+    assertEquals(3, logs.length);
+    for (int i = 0; i < logs.length; ++i) {
+      corruptLog(logs[i], 4);
+    }
+    verifyProcIdsOnRestart(procIds);
+  }
+
+  @Test
+  public void testCorruptedTrailer() throws Exception {
+    // Insert something
+    for (int i = 0; i < 100; ++i) {
+      procStore.insert(new TestSequentialProcedure(), null);
+    }
+
+    // Stop the store
+    procStore.stop(false);
+
+    // Remove 4 byte from the trailer
+    FileStatus[] logs = fs.listStatus(logDir);
+    assertEquals(1, logs.length);
+    corruptLog(logs[0], 4);
+
+    int count = countProcedures(storeRestart());
+    assertEquals(100, count);
+  }
+
+  @Test
+  public void testCorruptedEntries() throws Exception {
+    // Insert something
+    for (int i = 0; i < 100; ++i) {
+      procStore.insert(new TestSequentialProcedure(), null);
+    }
+
+    // Stop the store
+    procStore.stop(false);
+
+    // Remove some byte from the log
+    // (enough to cut the trailer and corrupt some entries)
+    FileStatus[] logs = fs.listStatus(logDir);
+    assertEquals(1, logs.length);
+    corruptLog(logs[0], 1823);
+
+    int count = countProcedures(storeRestart());
+    assertTrue(procStore.getCorruptedLogs() != null);
+    assertEquals(1, procStore.getCorruptedLogs().size());
+    assertEquals(85, count);
+  }
+
+  private void corruptLog(final FileStatus logFile, final long dropBytes)
+      throws IOException {
+    assertTrue(logFile.getLen() > dropBytes);
+    LOG.debug("corrupt log " + logFile.getPath() +
+              " size=" + logFile.getLen() + " drop=" + dropBytes);
+    Path tmpPath = new Path(testDir, "corrupted.log");
+    InputStream in = fs.open(logFile.getPath());
+    OutputStream out =  fs.create(tmpPath);
+    IOUtils.copyBytes(in, out, logFile.getLen() - dropBytes, true);
+    fs.rename(tmpPath, logFile.getPath());
+  }
+
+  private void verifyProcIdsOnRestart(final Set<Long> procIds) throws Exception {
+    int count = 0;
+    Iterator<Procedure> loader = storeRestart();
+    while (loader.hasNext()) {
+      Procedure proc = loader.next();
+      LOG.debug("loading procId=" + proc.getProcId());
+      assertTrue("procId=" + proc.getProcId() + " unexpected", procIds.contains(proc.getProcId()));
+      count++;
+    }
+    assertEquals(procIds.size(), count);
+  }
+
+  private void assertIsEmpty(Iterator<Procedure> iterator) {
+    assertEquals(0, countProcedures(iterator));
+  }
+
+  private int countProcedures(Iterator<Procedure> iterator) {
+    int count = 0;
+    while (iterator.hasNext()) {
+      Procedure proc = iterator.next();
+      LOG.trace("loading procId=" + proc.getProcId());
+      count++;
+    }
+    return count;
+  }
+
+  private void assertEmptyLogDir() {
+    try {
+      FileStatus[] status = fs.listStatus(logDir);
+      assertTrue("expected empty state-log dir", status == null || status.length == 0);
+    } catch (FileNotFoundException e) {
+      fail("expected the state-log dir to be present: " + logDir);
+    } catch (IOException e) {
+      fail("got en exception on state-log dir list: " + e.getMessage());
+    }
+  }
+
+  public static class TestSequentialProcedure extends SequentialProcedure<Void> {
+    private static long seqid = 0;
+
+    public TestSequentialProcedure() {
+      setProcId(++seqid);
+    }
+
+    @Override
+    protected Procedure[] execute(Void env) { return null; }
+
+    @Override
+    protected void rollback(Void env) { }
+
+    @Override
+    protected boolean abort(Void env) { return false; }
+
+    @Override
+    protected void serializeStateData(final OutputStream stream) throws IOException {
+      long procId = getProcId();
+      if (procId % 2 == 0) {
+        stream.write(Bytes.toBytes(procId));
+      }
+    }
+
+    @Override
+    protected void deserializeStateData(InputStream stream) throws IOException {
+      long procId = getProcId();
+      if (procId % 2 == 0) {
+        byte[] bProcId = new byte[8];
+        assertEquals(8, stream.read(bProcId));
+        assertEquals(procId, Bytes.toLong(bProcId));
+      } else {
+        assertEquals(0, stream.available());
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestTimeoutBlockingQueue.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestTimeoutBlockingQueue.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestTimeoutBlockingQueue.java
new file mode 100644
index 0000000..aff536a
--- /dev/null
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/util/TestTimeoutBlockingQueue.java
@@ -0,0 +1,137 @@
+/**
+ * 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.procedure2.util;
+
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+
+import org.junit.Assert;
+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 TestTimeoutBlockingQueue {
+  private static final Log LOG = LogFactory.getLog(TestTimeoutBlockingQueue.class);
+
+  static class TestObject {
+    private long timeout;
+    private int seqId;
+
+    public TestObject(int seqId, long timeout) {
+      this.timeout = timeout;
+      this.seqId = seqId;
+    }
+
+    public long getTimeout() {
+      return timeout;
+    }
+
+    public String toString() {
+      return String.format("(%03d, %03d)", seqId, timeout);
+    }
+  }
+
+  static class TestObjectTimeoutRetriever implements TimeoutRetriever<TestObject> {
+    @Override
+    public long getTimeout(TestObject obj) {
+      return obj.getTimeout();
+    }
+
+    @Override
+    public TimeUnit getTimeUnit(TestObject obj) {
+      return TimeUnit.MILLISECONDS;
+    }
+  }
+
+  @Test
+  public void testOrder() {
+    TimeoutBlockingQueue<TestObject> queue =
+      new TimeoutBlockingQueue<TestObject>(8, new TestObjectTimeoutRetriever());
+
+    long[] timeouts = new long[] {500, 200, 700, 300, 600, 600, 200, 800, 500};
+
+    for (int i = 0; i < timeouts.length; ++i) {
+      for (int j = 0; j <= i; ++j) {
+        queue.add(new TestObject(j, timeouts[j]));
+        queue.dump();
+      }
+
+      long prev = 0;
+      for (int j = 0; j <= i; ++j) {
+        TestObject obj = queue.poll();
+        assertTrue(obj.getTimeout() >= prev);
+        prev = obj.getTimeout();
+        queue.dump();
+      }
+    }
+  }
+
+  @Test
+  public void testTimeoutBlockingQueue() {
+    TimeoutBlockingQueue<TestObject> queue;
+
+    int[][] testArray = new int[][] {
+      {200, 400, 600},  // append
+      {200, 400, 100},  // prepend
+      {200, 400, 300},  // insert
+    };
+
+    for (int i = 0; i < testArray.length; ++i) {
+      int[] sortedArray = Arrays.copyOf(testArray[i], testArray[i].length);
+      Arrays.sort(sortedArray);
+
+      // test with head == 0
+      queue = new TimeoutBlockingQueue<TestObject>(2, new TestObjectTimeoutRetriever());
+      for (int j = 0; j < testArray[i].length; ++j) {
+        queue.add(new TestObject(j, testArray[i][j]));
+        queue.dump();
+      }
+
+      for (int j = 0; !queue.isEmpty(); ++j) {
+        assertEquals(sortedArray[j], queue.poll().getTimeout());
+      }
+
+      queue = new TimeoutBlockingQueue<TestObject>(2, new TestObjectTimeoutRetriever());
+      queue.add(new TestObject(0, 50));
+      assertEquals(50, queue.poll().getTimeout());
+
+      // test with head > 0
+      for (int j = 0; j < testArray[i].length; ++j) {
+        queue.add(new TestObject(j, testArray[i][j]));
+        queue.dump();
+      }
+
+      for (int j = 0; !queue.isEmpty(); ++j) {
+        assertEquals(sortedArray[j], queue.poll().getTimeout());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 7787c52..0d33332 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -176,6 +176,7 @@
                           <include>MapReduce.proto</include>
                           <include>Master.proto</include>
                           <include>MultiRowMutation.proto</include>
+                          <include>Procedure.proto</include>
                           <include>Quota.proto</include>
                           <include>RegionServerStatus.proto</include>
                           <include>RowProcessor.proto</include>


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

Branch: refs/heads/hbase-12439
Commit: c5bf41a7b15ce94b34dee263cbcba4d0c3b11c4c
Parents: b9c4b8f
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:21:18 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:06 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 4027 +++++++++++++++++-
 .../src/main/protobuf/MasterProcedure.proto     |   46 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   30 +-
 .../handler/TableDeleteFamilyHandler.java       |    6 +-
 .../procedure/AddColumnFamilyProcedure.java     |  409 ++
 .../procedure/DeleteColumnFamilyProcedure.java  |  441 ++
 .../procedure/ModifyColumnFamilyProcedure.java  |  384 ++
 .../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, 6323 insertions(+), 117 deletions(-)
----------------------------------------------------------------------



[14/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
HBASE-13202 Procedure v2 - core framework


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

Branch: refs/heads/hbase-12439
Commit: aaf77519845af028f774ea19d62a79136994a62e
Parents: 3cd929e
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 20:44:56 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:05 2015 +0100

----------------------------------------------------------------------
 .../hadoop/hbase/io/util/StreamUtils.java       |   12 +-
 .../hadoop/hbase/util/ForeignExceptionUtil.java |  109 +
 hbase-procedure/pom.xml                         |  181 +
 .../hbase/procedure2/OnePhaseProcedure.java     |   28 +
 .../hadoop/hbase/procedure2/Procedure.java      |  680 ++
 .../procedure2/ProcedureAbortedException.java   |   42 +
 .../hbase/procedure2/ProcedureException.java    |   45 +
 .../hbase/procedure2/ProcedureExecutor.java     | 1077 +++
 .../procedure2/ProcedureFairRunQueues.java      |  172 +
 .../hbase/procedure2/ProcedureResult.java       |   95 +
 .../hbase/procedure2/ProcedureRunnableSet.java  |   78 +
 .../procedure2/ProcedureSimpleRunQueue.java     |  121 +
 .../procedure2/ProcedureYieldException.java     |   40 +
 .../procedure2/RemoteProcedureException.java    |  116 +
 .../hbase/procedure2/RootProcedureState.java    |  185 +
 .../hbase/procedure2/SequentialProcedure.java   |   81 +
 .../hbase/procedure2/StateMachineProcedure.java |  166 +
 .../hbase/procedure2/TwoPhaseProcedure.java     |   28 +
 .../hbase/procedure2/store/ProcedureStore.java  |  121 +
 .../procedure2/store/ProcedureStoreTracker.java |  540 ++
 .../CorruptedWALProcedureStoreException.java    |   43 +
 .../procedure2/store/wal/ProcedureWALFile.java  |  152 +
 .../store/wal/ProcedureWALFormat.java           |  234 +
 .../store/wal/ProcedureWALFormatReader.java     |  166 +
 .../procedure2/store/wal/WALProcedureStore.java |  721 ++
 .../hadoop/hbase/procedure2/util/ByteSlot.java  |  111 +
 .../hbase/procedure2/util/StringUtils.java      |   80 +
 .../procedure2/util/TimeoutBlockingQueue.java   |  217 +
 .../procedure2/ProcedureTestingUtility.java     |  163 +
 .../procedure2/TestProcedureExecution.java      |  338 +
 .../procedure2/TestProcedureFairRunQueues.java  |  155 +
 .../hbase/procedure2/TestProcedureRecovery.java |  488 ++
 .../procedure2/TestProcedureReplayOrder.java    |  226 +
 .../store/TestProcedureStoreTracker.java        |  168 +
 .../store/wal/TestWALProcedureStore.java        |  267 +
 .../util/TestTimeoutBlockingQueue.java          |  137 +
 hbase-protocol/pom.xml                          |    1 +
 .../protobuf/generated/ProcedureProtos.java     | 7219 ++++++++++++++++++
 .../src/main/protobuf/Procedure.proto           |  114 +
 pom.xml                                         |   21 +-
 40 files changed, 14933 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
index 314ed2b..0b442a5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/util/StreamUtils.java
@@ -120,7 +120,7 @@ public class StreamUtils {
 
   /**
    * Reads a varInt value stored in an array.
-   * 
+   *
    * @param input
    *          Input array where the varInt is available
    * @param offset
@@ -198,4 +198,14 @@ public class StreamUtils {
     out.write((byte) (0xff & (v >> 8)));
     out.write((byte) (0xff & v));
   }
+
+  public static long readLong(InputStream in) throws IOException {
+    long result = 0;
+    for (int shift = 56; shift >= 0; shift -= 8) {
+      long x = in.read();
+      if (x < 0) throw new IOException("EOF");
+      result |= (x << shift);
+    }
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
new file mode 100644
index 0000000..a0006ed
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
@@ -0,0 +1,109 @@
+/**
+ * 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.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
+import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage;
+import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
+
+/**
+ * Helper to convert Exceptions and StackTraces from/to protobuf.
+ * (see ErrorHandling.proto for the internal of the proto messages)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class ForeignExceptionUtil {
+  private ForeignExceptionUtil() { }
+
+  public static IOException toIOException(final ForeignExceptionMessage eem) {
+    GenericExceptionMessage gem = eem.getGenericException();
+    StackTraceElement[] trace = toStackTrace(gem.getTraceList());
+    RemoteException re = new RemoteException(gem.getClassName(), gem.getMessage());
+    re.setStackTrace(trace);
+    return re.unwrapRemoteException();
+  }
+
+  public static ForeignExceptionMessage toProtoForeignException(String source, Throwable t) {
+    GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder();
+    gemBuilder.setClassName(t.getClass().getName());
+    if (t.getMessage() != null) {
+      gemBuilder.setMessage(t.getMessage());
+    }
+    // set the stack trace, if there is one
+    List<StackTraceElementMessage> stack = toProtoStackTraceElement(t.getStackTrace());
+    if (stack != null) {
+      gemBuilder.addAllTrace(stack);
+    }
+    GenericExceptionMessage payload = gemBuilder.build();
+    ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder();
+    exception.setGenericException(payload).setSource(source);
+    return exception.build();
+  }
+
+  /**
+   * Convert a stack trace to list of {@link StackTraceElement}.
+   * @param trace the stack trace to convert to protobuf message
+   * @return <tt>null</tt> if the passed stack is <tt>null</tt>.
+   */
+  public static List<StackTraceElementMessage> toProtoStackTraceElement(StackTraceElement[] trace) {
+    // if there is no stack trace, ignore it and just return the message
+    if (trace == null) return null;
+    // build the stack trace for the message
+    List<StackTraceElementMessage> pbTrace = new ArrayList<StackTraceElementMessage>(trace.length);
+    for (StackTraceElement elem : trace) {
+      StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder();
+      stackBuilder.setDeclaringClass(elem.getClassName());
+      if (elem.getFileName() != null) {
+        stackBuilder.setFileName(elem.getFileName());
+      }
+      stackBuilder.setLineNumber(elem.getLineNumber());
+      stackBuilder.setMethodName(elem.getMethodName());
+      pbTrace.add(stackBuilder.build());
+    }
+    return pbTrace;
+  }
+
+  /**
+   * Unwind a serialized array of {@link StackTraceElementMessage}s to a
+   * {@link StackTraceElement}s.
+   * @param traceList list that was serialized
+   * @return the deserialized list or <tt>null</tt> if it couldn't be unwound (e.g. wasn't set on
+   *         the sender).
+   */
+  public static StackTraceElement[] toStackTrace(List<StackTraceElementMessage> traceList) {
+    if (traceList == null || traceList.size() == 0) {
+      return new StackTraceElement[0]; // empty array
+    }
+    StackTraceElement[] trace = new StackTraceElement[traceList.size()];
+    for (int i = 0; i < traceList.size(); i++) {
+      StackTraceElementMessage elem = traceList.get(i);
+      trace[i] = new StackTraceElement(
+          elem.getDeclaringClass(), elem.getMethodName(),
+          elem.hasFileName() ? elem.getFileName() : null,
+          elem.getLineNumber());
+    }
+    return trace;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-procedure/pom.xml b/hbase-procedure/pom.xml
new file mode 100644
index 0000000..9683db2
--- /dev/null
+++ b/hbase-procedure/pom.xml
@@ -0,0 +1,181 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!--
+/**
+ * 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.
+ */
+-->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>hbase-procedure</artifactId>
+  <name>HBase - Procedure</name>
+  <description>Procedure Framework</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven.assembly.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <!-- Always skip the second part executions, since we only run
+        simple unit tests in this module. -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+    </dependency>
+   <dependency>
+     <groupId>org.apache.hbase</groupId>
+     <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Profiles for building against different hadoop versions -->
+    <profile>
+      <id>hadoop-1.1</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h1--><name>hadoop.profile</name><value>1.1</value>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+    <profile>
+      <id>hadoop-1.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>1.0</value>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+    <!--
+      profile for building against Hadoop 2.0.0-alpha. Activate using:
+       mvn -Dhadoop.profile=2.0
+    -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h2--><name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>3.0-SNAPSHOT</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/OnePhaseProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/OnePhaseProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/OnePhaseProcedure.java
new file mode 100644
index 0000000..1c3be2d
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/OnePhaseProcedure.java
@@ -0,0 +1,28 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public abstract class OnePhaseProcedure<TEnvironment> extends Procedure<TEnvironment> {
+  // TODO (e.g. used by online snapshots)
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
new file mode 100644
index 0000000..338fcad
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -0,0 +1,680 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+
+/**
+ * Base Procedure class responsible to handle the Procedure Metadata
+ * e.g. state, startTime, lastUpdate, stack-indexes, ...
+ *
+ * execute() is called each time the procedure is executed.
+ * it may be called multiple times in case of failure and restart, so the
+ * code must be idempotent.
+ * the return is a set of sub-procedures or null in case the procedure doesn't
+ * have sub-procedures. Once the sub-procedures are successfully completed
+ * the execute() method is called again, you should think at it as a stack:
+ *  -> step 1
+ *  ---> step 2
+ *  -> step 1
+ *
+ * rollback() is called when the procedure or one of the sub-procedures is failed.
+ * the rollback step is supposed to cleanup the resources created during the
+ * execute() step. in case of failure and restart rollback() may be called
+ * multiple times, so the code must be idempotent.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
+  // unchanged after initialization
+  private String owner = null;
+  private Long parentProcId = null;
+  private Long procId = null;
+  private long startTime;
+
+  // runtime state, updated every operation
+  private ProcedureState state = ProcedureState.INITIALIZING;
+  private Integer timeout = null;
+  private int[] stackIndexes = null;
+  private int childrenLatch = 0;
+  private long lastUpdate;
+
+  private RemoteProcedureException exception = null;
+  private byte[] result = null;
+
+  /**
+   * The main code of the procedure. It must be idempotent since execute()
+   * may be called multiple time in case of machine failure in the middle
+   * of the execution.
+   * @return a set of sub-procedures or null if there is nothing else to execute.
+   */
+  protected abstract Procedure[] execute(TEnvironment env)
+    throws ProcedureYieldException;
+
+  /**
+   * The code to undo what done by the execute() code.
+   * It is called when the procedure or one of the sub-procedure failed or an
+   * abort was requested. It should cleanup all the resources created by
+   * the execute() call. The implementation must be idempotent since rollback()
+   * may be called multiple time in case of machine failure in the middle
+   * of the execution.
+   * @throws IOException temporary failure, the rollback will retry later
+   */
+  protected abstract void rollback(TEnvironment env)
+    throws IOException;
+
+  /**
+   * The abort() call is asynchronous and each procedure must decide how to deal
+   * with that, if they want to be abortable. The simplest implementation
+   * is to have an AtomicBoolean set in the abort() method and then the execute()
+   * will check if the abort flag is set or not.
+   * abort() may be called multiple times from the client, so the implementation
+   * must be idempotent.
+   *
+   * NOTE: abort() is not like Thread.interrupt() it is just a notification
+   * that allows the procedure implementor where to abort to avoid leak and
+   * have a better control on what was executed and what not.
+   */
+  protected abstract boolean abort(TEnvironment env);
+
+  /**
+   * The user-level code of the procedure may have some state to
+   * persist (e.g. input arguments) to be able to resume on failure.
+   * @param stream the stream that will contain the user serialized data
+   */
+  protected abstract void serializeStateData(final OutputStream stream)
+    throws IOException;
+
+  /**
+   * Called on store load to allow the user to decode the previously serialized
+   * state.
+   * @param stream the stream that contains the user serialized data
+   */
+  protected abstract void deserializeStateData(final InputStream stream)
+    throws IOException;
+
+  /**
+   * The user should override this method, and try to take a lock if necessary.
+   * A lock can be anything, and it is up to the implementor.
+   * Example: in our Master we can execute request in parallel for different tables
+   *          create t1 and create t2 can be executed at the same time.
+   *          anything else on t1/t2 is queued waiting that specific table create to happen.
+   *
+   * @return true if the lock was acquired and false otherwise
+   */
+  protected boolean acquireLock(final TEnvironment env) {
+    return true;
+  }
+
+  /**
+   * The user should override this method, and release lock if necessary.
+   */
+  protected void releaseLock(final TEnvironment env) {
+    // no-op
+  }
+
+  /**
+   * Called when the procedure is loaded for replay.
+   * The procedure implementor may use this method to perform some quick
+   * operation before replay.
+   * e.g. failing the procedure if the state on replay may be unknown.
+   */
+  protected void beforeReplay(final TEnvironment env) {
+    // no-op
+  }
+
+  /**
+   * Called when the procedure is marked as completed (success or rollback).
+   * The procedure implementor may use this method to cleanup in-memory states.
+   * This operation will not be retried on failure.
+   */
+  protected void completionCleanup(final TEnvironment env) {
+    // no-op
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    toStringClassDetails(sb);
+
+    if (procId != null) {
+      sb.append(" id=");
+      sb.append(getProcId());
+    }
+
+    if (hasParent()) {
+      sb.append(" parent=");
+      sb.append(getParentProcId());
+    }
+
+    if (hasOwner()) {
+      sb.append(" owner=");
+      sb.append(getOwner());
+    }
+
+    sb.append(" state=");
+    sb.append(getState());
+    return sb.toString();
+  }
+
+  /**
+   * Extend the toString() information with the procedure details
+   * e.g. className and parameters
+   * @param builder the string builder to use to append the proc specific information
+   */
+  protected void toStringClassDetails(StringBuilder builder) {
+    builder.append(getClass().getName());
+  }
+
+  /**
+   * @return the serialized result if any, otherwise null
+   */
+  public byte[] getResult() {
+    return result;
+  }
+
+  /**
+   * The procedure may leave a "result" on completion.
+   * @param result the serialized result that will be passed to the client
+   */
+  protected void setResult(final byte[] result) {
+    this.result = result;
+  }
+
+  public long getProcId() {
+    return procId;
+  }
+
+  public boolean hasParent() {
+    return parentProcId != null;
+  }
+
+  public boolean hasException() {
+    return exception != null;
+  }
+
+  public boolean hasTimeout() {
+    return timeout != null;
+  }
+
+  public long getParentProcId() {
+    return parentProcId;
+  }
+
+  /**
+   * @return true if the procedure has failed.
+   *         true may mean failed but not yet rolledback or failed and rolledback.
+   */
+  public synchronized boolean isFailed() {
+    return exception != null || state == ProcedureState.ROLLEDBACK;
+  }
+
+  /**
+   * @return true if the procedure is finished successfully.
+   */
+  public synchronized boolean isSuccess() {
+    return state == ProcedureState.FINISHED && exception == null;
+  }
+
+  /**
+   * @return true if the procedure is finished. The Procedure may be completed
+   *         successfuly or failed and rolledback.
+   */
+  public synchronized boolean isFinished() {
+    switch (state) {
+      case ROLLEDBACK:
+        return true;
+      case FINISHED:
+        return exception == null;
+      default:
+        break;
+    }
+    return false;
+  }
+
+  /**
+   * @return true if the procedure is waiting for a child to finish or for an external event.
+   */
+  public synchronized boolean isWaiting() {
+    switch (state) {
+      case WAITING:
+      case WAITING_TIMEOUT:
+        return true;
+      default:
+        break;
+    }
+    return false;
+  }
+
+  public synchronized RemoteProcedureException getException() {
+    return exception;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public synchronized long getLastUpdate() {
+    return lastUpdate;
+  }
+
+  public synchronized long elapsedTime() {
+    return lastUpdate - startTime;
+  }
+
+  /**
+   * @param timeout timeout in msec
+   */
+  protected void setTimeout(final int timeout) {
+    this.timeout = timeout;
+  }
+
+  /**
+   * @return the timeout in msec
+   */
+  public int getTimeout() {
+    return timeout;
+  }
+
+  /**
+   * @return the remaining time before the timeout
+   */
+  public long getTimeRemaining() {
+    return Math.max(0, timeout - (EnvironmentEdgeManager.currentTime() - startTime));
+  }
+
+  protected void setOwner(final String owner) {
+    this.owner = StringUtils.isEmpty(owner) ? null : owner;
+  }
+
+  public String getOwner() {
+    return owner;
+  }
+
+  public boolean hasOwner() {
+    return owner != null;
+  }
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  protected synchronized void setState(final ProcedureState state) {
+    this.state = state;
+    updateTimestamp();
+  }
+
+  @InterfaceAudience.Private
+  protected synchronized ProcedureState getState() {
+    return state;
+  }
+
+  protected void setFailure(final String source, final Throwable cause) {
+    setFailure(new RemoteProcedureException(source, cause));
+  }
+
+  protected synchronized void setFailure(final RemoteProcedureException exception) {
+    this.exception = exception;
+    if (!isFinished()) {
+      setState(ProcedureState.FINISHED);
+    }
+  }
+
+  protected void setAbortFailure(final String source, final String msg) {
+    setFailure(source, new ProcedureAbortedException(msg));
+  }
+
+  @InterfaceAudience.Private
+  protected synchronized boolean setTimeoutFailure() {
+    if (state == ProcedureState.WAITING_TIMEOUT) {
+      long timeDiff = EnvironmentEdgeManager.currentTime() - lastUpdate;
+      setFailure("ProcedureExecutor", new TimeoutException(
+        "Operation timed out after " + StringUtils.humanTimeDiff(timeDiff)));
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to assign the ID to the newly created procedure.
+   */
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  protected void setProcId(final long procId) {
+    this.procId = procId;
+    this.startTime = EnvironmentEdgeManager.currentTime();
+    setState(ProcedureState.RUNNABLE);
+  }
+
+  /**
+   * Called by the ProcedureExecutor to assign the parent to the newly created procedure.
+   */
+  @InterfaceAudience.Private
+  protected void setParentProcId(final long parentProcId) {
+    this.parentProcId = parentProcId;
+  }
+
+  /**
+   * Internal method called by the ProcedureExecutor that starts the
+   * user-level code execute().
+   */
+  @InterfaceAudience.Private
+  protected Procedure[] doExecute(final TEnvironment env)
+      throws ProcedureYieldException {
+    try {
+      updateTimestamp();
+      return execute(env);
+    } finally {
+      updateTimestamp();
+    }
+  }
+
+  /**
+   * Internal method called by the ProcedureExecutor that starts the
+   * user-level code rollback().
+   */
+  @InterfaceAudience.Private
+  protected void doRollback(final TEnvironment env) throws IOException {
+    try {
+      updateTimestamp();
+      rollback(env);
+    } finally {
+      updateTimestamp();
+    }
+  }
+
+  /**
+   * Called on store load to initialize the Procedure internals after
+   * the creation/deserialization.
+   */
+  @InterfaceAudience.Private
+  protected void setStartTime(final long startTime) {
+    this.startTime = startTime;
+  }
+
+  /**
+   * Called on store load to initialize the Procedure internals after
+   * the creation/deserialization.
+   */
+  private synchronized void setLastUpdate(final long lastUpdate) {
+    this.lastUpdate = lastUpdate;
+  }
+
+  protected synchronized void updateTimestamp() {
+    this.lastUpdate = EnvironmentEdgeManager.currentTime();
+  }
+
+  /**
+   * Called by the ProcedureExecutor on procedure-load to restore the latch state
+   */
+  @InterfaceAudience.Private
+  protected synchronized void setChildrenLatch(final int numChildren) {
+    this.childrenLatch = numChildren;
+  }
+
+  /**
+   * Called by the ProcedureExecutor on procedure-load to restore the latch state
+   */
+  @InterfaceAudience.Private
+  protected synchronized void incChildrenLatch() {
+    // TODO: can this be inferred from the stack? I think so...
+    this.childrenLatch++;
+  }
+
+  /**
+   * Called by the ProcedureExecutor to notify that one of the sub-procedures
+   * has completed.
+   */
+  @InterfaceAudience.Private
+  protected synchronized boolean childrenCountDown() {
+    assert childrenLatch > 0;
+    return --childrenLatch == 0;
+  }
+
+  /**
+   * Called by the RootProcedureState on procedure execution.
+   * Each procedure store its stack-index positions.
+   */
+  @InterfaceAudience.Private
+  protected synchronized void addStackIndex(final int index) {
+    if (stackIndexes == null) {
+      stackIndexes = new int[] { index };
+    } else {
+      int count = stackIndexes.length;
+      stackIndexes = Arrays.copyOf(stackIndexes, count + 1);
+      stackIndexes[count] = index;
+    }
+  }
+
+  @InterfaceAudience.Private
+  protected synchronized boolean removeStackIndex() {
+    if (stackIndexes.length > 1) {
+      stackIndexes = Arrays.copyOf(stackIndexes, stackIndexes.length - 1);
+      return false;
+    } else {
+      stackIndexes = null;
+      return true;
+    }
+  }
+
+  /**
+   * Called on store load to initialize the Procedure internals after
+   * the creation/deserialization.
+   */
+  @InterfaceAudience.Private
+  protected synchronized void setStackIndexes(final List<Integer> stackIndexes) {
+    this.stackIndexes = new int[stackIndexes.size()];
+    for (int i = 0; i < this.stackIndexes.length; ++i) {
+      this.stackIndexes[i] = stackIndexes.get(i);
+    }
+  }
+
+  @InterfaceAudience.Private
+  protected synchronized boolean wasExecuted() {
+    return stackIndexes != null;
+  }
+
+  @InterfaceAudience.Private
+  protected synchronized int[] getStackIndexes() {
+    return stackIndexes;
+  }
+
+  @Override
+  public int compareTo(final Procedure other) {
+    long diff = getProcId() - other.getProcId();
+    return (diff < 0) ? -1 : (diff > 0) ? 1 : 0;
+  }
+
+  /*
+   * Helper to lookup the root Procedure ID given a specified procedure.
+   */
+  @InterfaceAudience.Private
+  protected static Long getRootProcedureId(final Map<Long, Procedure> procedures, Procedure proc) {
+    while (proc.hasParent()) {
+      proc = procedures.get(proc.getParentProcId());
+      if (proc == null) return null;
+    }
+    return proc.getProcId();
+  }
+
+  protected static Procedure newInstance(final String className) throws IOException {
+    try {
+      Class<?> clazz = Class.forName(className);
+      if (!Modifier.isPublic(clazz.getModifiers())) {
+        throw new Exception("the " + clazz + " class is not public");
+      }
+
+      Constructor<?> ctor = clazz.getConstructor();
+      assert ctor != null : "no constructor found";
+      if (!Modifier.isPublic(ctor.getModifiers())) {
+        throw new Exception("the " + clazz + " constructor is not public");
+      }
+      return (Procedure)ctor.newInstance();
+    } catch (Exception e) {
+      throw new IOException("The procedure class " + className +
+          " must be accessible and have an empty constructor", e);
+    }
+  }
+
+  protected static void validateClass(final Procedure proc) throws IOException {
+    try {
+      Class<?> clazz = proc.getClass();
+      if (!Modifier.isPublic(clazz.getModifiers())) {
+        throw new Exception("the " + clazz + " class is not public");
+      }
+
+      Constructor<?> ctor = clazz.getConstructor();
+      assert ctor != null;
+      if (!Modifier.isPublic(ctor.getModifiers())) {
+        throw new Exception("the " + clazz + " constructor is not public");
+      }
+    } catch (Exception e) {
+      throw new IOException("The procedure class " + proc.getClass().getName() +
+          " must be accessible and have an empty constructor", e);
+    }
+  }
+
+  /**
+   * Helper to convert the procedure to protobuf.
+   * Used by ProcedureStore implementations.
+   */
+  @InterfaceAudience.Private
+  public static ProcedureProtos.Procedure convert(final Procedure proc)
+      throws IOException {
+    Preconditions.checkArgument(proc != null);
+    validateClass(proc);
+
+    ProcedureProtos.Procedure.Builder builder = ProcedureProtos.Procedure.newBuilder()
+      .setClassName(proc.getClass().getName())
+      .setProcId(proc.getProcId())
+      .setState(proc.getState())
+      .setStartTime(proc.getStartTime())
+      .setLastUpdate(proc.getLastUpdate());
+
+    if (proc.hasParent()) {
+      builder.setParentId(proc.getParentProcId());
+    }
+
+    if (proc.hasTimeout()) {
+      builder.setTimeout(proc.getTimeout());
+    }
+
+    if (proc.hasOwner()) {
+      builder.setOwner(proc.getOwner());
+    }
+
+    int[] stackIds = proc.getStackIndexes();
+    if (stackIds != null) {
+      for (int i = 0; i < stackIds.length; ++i) {
+        builder.addStackId(stackIds[i]);
+      }
+    }
+
+    if (proc.hasException()) {
+      RemoteProcedureException exception = proc.getException();
+      builder.setException(
+        RemoteProcedureException.toProto(exception.getSource(), exception.getCause()));
+    }
+
+    byte[] result = proc.getResult();
+    if (result != null) {
+      builder.setResult(ByteStringer.wrap(result));
+    }
+
+    ByteString.Output stateStream = ByteString.newOutput();
+    proc.serializeStateData(stateStream);
+    if (stateStream.size() > 0) {
+      builder.setStateData(stateStream.toByteString());
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Helper to convert the protobuf procedure.
+   * Used by ProcedureStore implementations.
+   *
+   * TODO: OPTIMIZATION: some of the field never change during the execution
+   *                     (e.g. className, procId, parentId, ...).
+   *                     We can split in 'data' and 'state', and the store
+   *                     may take advantage of it by storing the data only on insert().
+   */
+  @InterfaceAudience.Private
+  public static Procedure convert(final ProcedureProtos.Procedure proto)
+      throws IOException {
+    // Procedure from class name
+    Procedure proc = Procedure.newInstance(proto.getClassName());
+
+    // set fields
+    proc.setProcId(proto.getProcId());
+    proc.setState(proto.getState());
+    proc.setStartTime(proto.getStartTime());
+    proc.setLastUpdate(proto.getLastUpdate());
+
+    if (proto.hasParentId()) {
+      proc.setParentProcId(proto.getParentId());
+    }
+
+    if (proto.hasOwner()) {
+      proc.setOwner(proto.getOwner());
+    }
+
+    if (proto.hasTimeout()) {
+      proc.setTimeout(proto.getTimeout());
+    }
+
+    if (proto.getStackIdCount() > 0) {
+      proc.setStackIndexes(proto.getStackIdList());
+    }
+
+    if (proto.hasException()) {
+      assert proc.getState() == ProcedureState.FINISHED ||
+             proc.getState() == ProcedureState.ROLLEDBACK :
+             "The procedure must be failed (waiting to rollback) or rolledback";
+      proc.setFailure(RemoteProcedureException.fromProto(proto.getException()));
+    }
+
+    if (proto.hasResult()) {
+      proc.setResult(proto.getResult().toByteArray());
+    }
+
+    // we want to call deserialize even when the stream is empty, mainly for testing.
+    proc.deserializeStateData(proto.getStateData().newInput());
+
+    return proc;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureAbortedException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureAbortedException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureAbortedException.java
new file mode 100644
index 0000000..2e409cf
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureAbortedException.java
@@ -0,0 +1,42 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Thrown when a procedure is aborted
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ProcedureAbortedException extends ProcedureException {
+  /** default constructor */
+  public ProcedureAbortedException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public ProcedureAbortedException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureException.java
new file mode 100644
index 0000000..9f922b1
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureException.java
@@ -0,0 +1,45 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ProcedureException extends IOException {
+  /** default constructor */
+  public ProcedureException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public ProcedureException(String s) {
+    super(s);
+  }
+
+  public ProcedureException(Throwable t) {
+    super(t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
new file mode 100644
index 0000000..2982058
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -0,0 +1,1077 @@
+/**
+ * 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.procedure2;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.HashSet;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue;
+import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Thread Pool that executes the submitted procedures.
+ * The executor has a ProcedureStore associated.
+ * Each operation is logged and on restart the pending procedures are resumed.
+ *
+ * Unless the Procedure code throws an error (e.g. invalid user input)
+ * the procedure will complete (at some point in time), On restart the pending
+ * procedures are resumed and the once failed will be rolledback.
+ *
+ * The user can add procedures to the executor via submitProcedure(proc)
+ * check for the finished state via isFinished(procId)
+ * and get the result via getResult(procId)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureExecutor<TEnvironment> {
+  private static final Log LOG = LogFactory.getLog(ProcedureExecutor.class);
+
+  Testing testing = null;
+  public static class Testing {
+    protected boolean killBeforeStoreUpdate = false;
+    protected boolean toggleKillBeforeStoreUpdate = false;
+
+    protected boolean shouldKillBeforeStoreUpdate() {
+      final boolean kill = this.killBeforeStoreUpdate;
+      if (this.toggleKillBeforeStoreUpdate) {
+        this.killBeforeStoreUpdate = !kill;
+        LOG.warn("Toggle Kill before store update to: " + this.killBeforeStoreUpdate);
+      }
+      return kill;
+    }
+  }
+
+  public interface ProcedureExecutorListener {
+    void procedureLoaded(long procId);
+    void procedureAdded(long procId);
+    void procedureFinished(long procId);
+  }
+
+  /**
+   * Used by the TimeoutBlockingQueue to get the timeout interval of the procedure
+   */
+  private static class ProcedureTimeoutRetriever implements TimeoutRetriever<Procedure> {
+    @Override
+    public long getTimeout(Procedure proc) {
+      return proc.getTimeRemaining();
+    }
+
+    @Override
+    public TimeUnit getTimeUnit(Procedure proc) {
+      return TimeUnit.MILLISECONDS;
+    }
+  }
+
+  /**
+   * Internal cleaner that removes the completed procedure results after a TTL.
+   * NOTE: This is a special case handled in timeoutLoop().
+   *
+   * Since the client code looks more or less like:
+   *   procId = master.doOperation()
+   *   while (master.getProcResult(procId) == ProcInProgress);
+   * The master should not throw away the proc result as soon as the procedure is done
+   * but should wait a result request from the client (see executor.removeResult(procId))
+   * The client will call something like master.isProcDone() or master.getProcResult()
+   * which will return the result/state to the client, and it will mark the completed
+   * proc as ready to delete. note that the client may not receive the response from
+   * the master (e.g. master failover) so, if we delay a bit the real deletion of
+   * the proc result the client will be able to get the result the next try.
+   */
+  private static class CompletedProcedureCleaner<TEnvironment> extends Procedure<TEnvironment> {
+    private static final Log LOG = LogFactory.getLog(CompletedProcedureCleaner.class);
+
+    private static final String CLEANER_INTERVAL_CONF_KEY = "hbase.procedure.cleaner.interval";
+    private static final int DEFAULT_CLEANER_INTERVAL = 30 * 1000; // 30sec
+
+    private static final String EVICT_TTL_CONF_KEY = "hbase.procedure.cleaner.evict.ttl";
+    private static final int DEFAULT_EVICT_TTL = 15 * 60000; // 15min
+
+    private static final String EVICT_ACKED_TTL_CONF_KEY ="hbase.procedure.cleaner.acked.evict.ttl";
+    private static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min
+
+    private final Map<Long, ProcedureResult> completed;
+    private final ProcedureStore store;
+    private final Configuration conf;
+
+    public CompletedProcedureCleaner(final Configuration conf, final ProcedureStore store,
+        final Map<Long, ProcedureResult> completedMap) {
+      // set the timeout interval that triggers the periodic-procedure
+      setTimeout(conf.getInt(CLEANER_INTERVAL_CONF_KEY, DEFAULT_CLEANER_INTERVAL));
+      this.completed = completedMap;
+      this.store = store;
+      this.conf = conf;
+    }
+
+    public void periodicExecute(final TEnvironment env) {
+      if (completed.isEmpty()) {
+        LOG.debug("no completed procedures to cleanup");
+        return;
+      }
+
+      final long evictTtl = conf.getInt(EVICT_TTL_CONF_KEY, DEFAULT_EVICT_TTL);
+      final long evictAckTtl = conf.getInt(EVICT_ACKED_TTL_CONF_KEY, DEFAULT_ACKED_EVICT_TTL);
+
+      long now = EnvironmentEdgeManager.currentTime();
+      Iterator<Map.Entry<Long, ProcedureResult>> it = completed.entrySet().iterator();
+      while (it.hasNext() && store.isRunning()) {
+        Map.Entry<Long, ProcedureResult> entry = it.next();
+        ProcedureResult result = entry.getValue();
+
+        // TODO: Select TTL based on Procedure type
+        if ((result.hasClientAckTime() && (now - result.getClientAckTime()) >= evictAckTtl) ||
+            (now - result.getLastUpdate()) >= evictTtl) {
+          LOG.debug("Evict completed procedure " + entry.getKey());
+          store.delete(entry.getKey());
+          it.remove();
+        }
+      }
+    }
+
+    @Override
+    protected Procedure[] execute(final TEnvironment env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected void rollback(final TEnvironment env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected boolean abort(final TEnvironment env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void serializeStateData(final OutputStream stream) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void deserializeStateData(final InputStream stream) {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  /**
+   * Map the the procId returned by submitProcedure(), the Root-ProcID, to the ProcedureResult.
+   * Once a Root-Procedure completes (success or failure), the result will be added to this map.
+   * The user of ProcedureExecutor should call getResult(procId) to get the result.
+   */
+  private final ConcurrentHashMap<Long, ProcedureResult> completed =
+    new ConcurrentHashMap<Long, ProcedureResult>();
+
+  /**
+   * Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState.
+   * The RootProcedureState contains the execution stack of the Root-Procedure,
+   * It is added to the map by submitProcedure() and removed on procedure completion.
+   */
+  private final ConcurrentHashMap<Long, RootProcedureState> rollbackStack =
+    new ConcurrentHashMap<Long, RootProcedureState>();
+
+  /**
+   * Helper map to lookup the live procedures by ID.
+   * This map contains every procedure. root-procedures and subprocedures.
+   */
+  private final ConcurrentHashMap<Long, Procedure> procedures =
+    new ConcurrentHashMap<Long, Procedure>();
+
+  /**
+   * Timeout Queue that contains Procedures in a WAITING_TIMEOUT state
+   * or periodic procedures.
+   */
+  private final TimeoutBlockingQueue<Procedure> waitingTimeout =
+    new TimeoutBlockingQueue<Procedure>(new ProcedureTimeoutRetriever());
+
+  /**
+   * Queue that contains runnable procedures.
+   */
+  private final ProcedureRunnableSet runnables;
+
+  // TODO
+  private final ReentrantLock submitLock = new ReentrantLock();
+  private final AtomicLong lastProcId = new AtomicLong(-1);
+
+  private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners =
+    new CopyOnWriteArrayList<ProcedureExecutorListener>();
+
+  private final AtomicInteger activeExecutorCount = new AtomicInteger(0);
+  private final AtomicBoolean running = new AtomicBoolean(false);
+  private final TEnvironment environment;
+  private final ProcedureStore store;
+  private final Configuration conf;
+
+  private Thread[] threads;
+
+  public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
+      final ProcedureStore store) {
+    this(conf, environment, store, new ProcedureSimpleRunQueue());
+  }
+
+  public ProcedureExecutor(final Configuration conf, final TEnvironment environment,
+      final ProcedureStore store, final ProcedureRunnableSet runqueue) {
+    this.environment = environment;
+    this.runnables = runqueue;
+    this.store = store;
+    this.conf = conf;
+  }
+
+  private List<Map.Entry<Long, RootProcedureState>> load() throws IOException {
+    Preconditions.checkArgument(completed.isEmpty());
+    Preconditions.checkArgument(rollbackStack.isEmpty());
+    Preconditions.checkArgument(procedures.isEmpty());
+    Preconditions.checkArgument(waitingTimeout.isEmpty());
+    Preconditions.checkArgument(runnables.size() == 0);
+
+    // 1. Load the procedures
+    Iterator<Procedure> loader = store.load();
+    if (loader == null) {
+      lastProcId.set(0);
+      return null;
+    }
+
+    long logMaxProcId = 0;
+    int runnablesCount = 0;
+    while (loader.hasNext()) {
+      Procedure proc = loader.next();
+      proc.beforeReplay(getEnvironment());
+      procedures.put(proc.getProcId(), proc);
+      logMaxProcId = Math.max(logMaxProcId, proc.getProcId());
+      LOG.debug("Loading procedure state=" + proc.getState() +
+                " isFailed=" + proc.hasException() + ": " + proc);
+      if (!proc.hasParent() && !proc.isFinished()) {
+        rollbackStack.put(proc.getProcId(), new RootProcedureState());
+      }
+      if (proc.getState() == ProcedureState.RUNNABLE) {
+        runnablesCount++;
+      }
+    }
+    assert lastProcId.get() < 0;
+    lastProcId.set(logMaxProcId);
+
+    // 2. Initialize the stacks
+    TreeSet<Procedure> runnableSet = null;
+    HashSet<Procedure> waitingSet = null;
+    for (final Procedure proc: procedures.values()) {
+      Long rootProcId = getRootProcedureId(proc);
+      if (rootProcId == null) {
+        // The 'proc' was ready to run but the root procedure was rolledback?
+        runnables.addBack(proc);
+        continue;
+      }
+
+      if (!proc.hasParent() && proc.isFinished()) {
+        LOG.debug("The procedure is completed state=" + proc.getState() +
+                  " isFailed=" + proc.hasException() + ": " + proc);
+        assert !rollbackStack.containsKey(proc.getProcId());
+        completed.put(proc.getProcId(), newResultFromProcedure(proc));
+        continue;
+      }
+
+      if (proc.hasParent() && !proc.isFinished()) {
+        Procedure parent = procedures.get(proc.getParentProcId());
+        // corrupted procedures are handled later at step 3
+        if (parent != null) {
+          parent.incChildrenLatch();
+        }
+      }
+
+      RootProcedureState procStack = rollbackStack.get(rootProcId);
+      procStack.loadStack(proc);
+
+      switch (proc.getState()) {
+        case RUNNABLE:
+          if (runnableSet == null) {
+            runnableSet = new TreeSet<Procedure>();
+          }
+          runnableSet.add(proc);
+          break;
+        case WAITING_TIMEOUT:
+          if (waitingSet == null) {
+            waitingSet = new HashSet<Procedure>();
+          }
+          waitingSet.add(proc);
+          break;
+        case FINISHED:
+          if (proc.hasException()) {
+            // add the proc to the runnables to perform the rollback
+            runnables.addBack(proc);
+            break;
+          }
+        case ROLLEDBACK:
+        case INITIALIZING:
+          String msg = "Unexpected " + proc.getState() + " state for " + proc;
+          LOG.error(msg);
+          throw new UnsupportedOperationException(msg);
+        default:
+          break;
+      }
+    }
+
+    // 3. Validate the stacks
+    List<Map.Entry<Long, RootProcedureState>> corrupted = null;
+    Iterator<Map.Entry<Long, RootProcedureState>> itStack = rollbackStack.entrySet().iterator();
+    while (itStack.hasNext()) {
+      Map.Entry<Long, RootProcedureState> entry = itStack.next();
+      RootProcedureState procStack = entry.getValue();
+      if (procStack.isValid()) continue;
+
+      for (Procedure proc: procStack.getSubprocedures()) {
+        procedures.remove(proc.getProcId());
+        if (runnableSet != null) runnableSet.remove(proc);
+        if (waitingSet != null) waitingSet.remove(proc);
+      }
+      itStack.remove();
+      if (corrupted == null) {
+        corrupted = new ArrayList<Map.Entry<Long, RootProcedureState>>();
+      }
+      corrupted.add(entry);
+    }
+
+    // 4. Push the runnables
+    if (runnableSet != null) {
+      // TODO: See ProcedureWALFormatReader.readInitEntry() some procedure
+      // may be started way before this stuff.
+      for (Procedure proc: runnableSet) {
+        if (!proc.hasParent()) {
+          sendProcedureLoadedNotification(proc.getProcId());
+        }
+        runnables.addBack(proc);
+      }
+    }
+    return corrupted;
+  }
+
+  public void start(int numThreads) throws IOException {
+    if (running.getAndSet(true)) {
+      LOG.warn("Already running");
+      return;
+    }
+
+    // We have numThreads executor + one timer thread used for timing out
+    // procedures and triggering periodic procedures.
+    threads = new Thread[numThreads + 1];
+    LOG.info("Starting procedure executor threads=" + threads.length);
+
+    // Initialize procedures executor
+    for (int i = 0; i < numThreads; ++i) {
+      threads[i] = new Thread("ProcedureExecutorThread-" + i) {
+        @Override
+        public void run() {
+          execLoop();
+        }
+      };
+    }
+
+    // Initialize procedures timeout handler (this is the +1 thread)
+    threads[numThreads] = new Thread("ProcedureExecutorTimeoutThread") {
+      @Override
+      public void run() {
+        timeoutLoop();
+      }
+    };
+
+    // Acquire the store lease.
+    store.recoverLease();
+
+    // TODO: Split in two steps.
+    // TODO: Handle corrupted procedure returned (probably just a WARN)
+    // The first one will make sure that we have the latest id,
+    // so we can start the threads and accept new procedures.
+    // The second step will do the actual load of old procedures.
+    load();
+
+    // Start the executors. Here we must have the lastProcId set.
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].start();
+    }
+
+    // Add completed cleaner
+    waitingTimeout.add(new CompletedProcedureCleaner(conf, store, completed));
+  }
+
+  public void stop() {
+    if (!running.getAndSet(false)) {
+      return;
+    }
+
+    LOG.info("Stopping the procedure executor");
+    runnables.signalAll();
+    waitingTimeout.signalAll();
+  }
+
+  public void join() {
+    boolean interrupted = false;
+
+    for (int i = 0; i < threads.length; ++i) {
+      try {
+        threads[i].join();
+      } catch (InterruptedException ex) {
+        interrupted = true;
+      }
+    }
+
+    if (interrupted) {
+      Thread.currentThread().interrupt();
+    }
+
+    completed.clear();
+    rollbackStack.clear();
+    procedures.clear();
+    waitingTimeout.clear();
+    runnables.clear();
+    lastProcId.set(-1);
+  }
+
+  public boolean isRunning() {
+    return running.get();
+  }
+
+  /**
+   * @return the number of execution threads.
+   */
+  public int getNumThreads() {
+    return threads == null ? 0 : (threads.length - 1);
+  }
+
+  public int getActiveExecutorCount() {
+    return activeExecutorCount.get();
+  }
+
+  public TEnvironment getEnvironment() {
+    return this.environment;
+  }
+
+  public ProcedureStore getStore() {
+    return this.store;
+  }
+
+  public void registerListener(ProcedureExecutorListener listener) {
+    this.listeners.add(listener);
+  }
+
+  public boolean unregisterListener(ProcedureExecutorListener listener) {
+    return this.listeners.remove(listener);
+  }
+
+  /**
+   * Add a new root-procedure to the executor.
+   * @param proc the new procedure to execute.
+   * @return the procedure id, that can be used to monitor the operation
+   */
+  public long submitProcedure(final Procedure proc) {
+    Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING);
+    Preconditions.checkArgument(isRunning());
+    Preconditions.checkArgument(lastProcId.get() >= 0);
+    Preconditions.checkArgument(!proc.hasParent());
+
+    // Initialize the Procedure ID
+    proc.setProcId(nextProcId());
+
+    // Commit the transaction
+    store.insert(proc, null);
+    LOG.debug("procedure " + proc + " added to the store");
+
+    // Create the rollback stack for the procedure
+    RootProcedureState stack = new RootProcedureState();
+    rollbackStack.put(proc.getProcId(), stack);
+
+    // Submit the new subprocedures
+    assert !procedures.containsKey(proc.getProcId());
+    procedures.put(proc.getProcId(), proc);
+    sendProcedureAddedNotification(proc.getProcId());
+    runnables.addBack(proc);
+    return proc.getProcId();
+  }
+
+  public ProcedureResult getResult(final long procId) {
+    return completed.get(procId);
+  }
+
+  /**
+   * Return true if the procedure is finished.
+   * The state may be "completed successfully" or "failed and rolledback".
+   * Use getResult() to check the state or get the result data.
+   * @param procId the ID of the procedure to check
+   * @return true if the procedure execution is finished, otherwise false.
+   */
+  public boolean isFinished(final long procId) {
+    return completed.containsKey(procId);
+  }
+
+  /**
+   * Return true if the procedure is started.
+   * @param procId the ID of the procedure to check
+   * @return true if the procedure execution is started, otherwise false.
+   */
+  public boolean isStarted(final long procId) {
+    Procedure proc = procedures.get(procId);
+    if (proc == null) {
+      return completed.get(procId) != null;
+    }
+    return proc.wasExecuted();
+  }
+
+  /**
+   * Mark the specified completed procedure, as ready to remove.
+   * @param procId the ID of the procedure to remove
+   */
+  public void removeResult(final long procId) {
+    ProcedureResult result = completed.get(procId);
+    if (result == null) {
+      assert !procedures.containsKey(procId) : "procId=" + procId + " is still running";
+      LOG.debug("Procedure procId=" + procId + " already removed by the cleaner");
+      return;
+    }
+
+    // The CompletedProcedureCleaner will take care of deletion, once the TTL is expired.
+    result.setClientAckTime(EnvironmentEdgeManager.currentTime());
+  }
+
+  /**
+   * Send an abort notification the specified procedure.
+   * Depending on the procedure implementation the abort can be considered or ignored.
+   * @param procId the procedure to abort
+   * @return true if the procedure exist and has received the abort, otherwise false.
+   */
+  public boolean abort(final long procId) {
+    Procedure proc = procedures.get(procId);
+    if (proc != null) {
+      return proc.abort(getEnvironment());
+    }
+    return false;
+  }
+
+  public Map<Long, ProcedureResult> getResults() {
+    return Collections.unmodifiableMap(completed);
+  }
+
+  public Procedure getProcedure(final long procId) {
+    return procedures.get(procId);
+  }
+
+  protected ProcedureRunnableSet getRunnableSet() {
+    return runnables;
+  }
+
+  /**
+   * Execution loop (N threads)
+   * while the executor is in a running state,
+   * fetch a procedure from the runnables queue and start the execution.
+   */
+  private void execLoop() {
+    while (isRunning()) {
+      Long procId = runnables.poll();
+      Procedure proc = procId != null ? procedures.get(procId) : null;
+      if (proc == null) continue;
+
+      try {
+        activeExecutorCount.incrementAndGet();
+        execLoop(proc);
+      } finally {
+        activeExecutorCount.decrementAndGet();
+      }
+    }
+  }
+
+  private void execLoop(Procedure proc) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("trying to start the execution of " + proc);
+    }
+
+    Long rootProcId = getRootProcedureId(proc);
+    if (rootProcId == null) {
+      // The 'proc' was ready to run but the root procedure was rolledback
+      executeRollback(proc);
+      return;
+    }
+
+    RootProcedureState procStack = rollbackStack.get(rootProcId);
+    if (procStack == null) return;
+
+    do {
+      // Try to acquire the execution
+      if (!procStack.acquire(proc)) {
+        if (procStack.setRollback()) {
+          // we have the 'rollback-lock' we can start rollingback
+          if (!executeRollback(rootProcId, procStack)) {
+            procStack.unsetRollback();
+            runnables.yield(proc);
+          }
+        } else {
+          // if we can't rollback means that some child is still running.
+          // the rollback will be executed after all the children are done.
+          // If the procedure was never executed, remove and mark it as rolledback.
+          if (!proc.wasExecuted()) {
+            if (!executeRollback(proc)) {
+              runnables.yield(proc);
+            }
+          }
+        }
+        break;
+      }
+
+      // Execute the procedure
+      assert proc.getState() == ProcedureState.RUNNABLE;
+      if (proc.acquireLock(getEnvironment())) {
+        execProcedure(procStack, proc);
+        proc.releaseLock(getEnvironment());
+      } else {
+        runnables.yield(proc);
+      }
+      procStack.release(proc);
+
+      // allows to kill the executor before something is stored to the wal.
+      // useful to test the procedure recovery.
+      if (testing != null && !isRunning()) {
+        break;
+      }
+
+      if (proc.getProcId() == rootProcId && proc.isSuccess()) {
+        // Finalize the procedure state
+        LOG.info("Procedure completed in " +
+            StringUtils.humanTimeDiff(proc.elapsedTime()) + ": " + proc);
+        procedureFinished(proc);
+        break;
+      }
+    } while (procStack.isFailed());
+  }
+
+  private void timeoutLoop() {
+    while (isRunning()) {
+      Procedure proc = waitingTimeout.poll();
+      if (proc == null) continue;
+
+      if (proc.getTimeRemaining() > 100) {
+        // got an early wake, maybe a stop?
+        // re-enqueue the task in case was not a stop or just a signal
+        waitingTimeout.add(proc);
+        continue;
+      }
+
+      // ----------------------------------------------------------------------------
+      // TODO-MAYBE: Should we provide a notification to the store with the
+      // full set of procedures pending and completed to write a compacted
+      // version of the log (in case is a log)?
+      // In theory no, procedures are have a short life, so at some point the store
+      // will have the tracker saying everything is in the last log.
+      // ----------------------------------------------------------------------------
+
+      // The CompletedProcedureCleaner is a special case, and it acts as a chore.
+      // instead of bringing the Chore class in, we reuse this timeout thread for
+      // this special case.
+      if (proc instanceof CompletedProcedureCleaner) {
+        try {
+          ((CompletedProcedureCleaner)proc).periodicExecute(getEnvironment());
+        } catch (Throwable e) {
+          LOG.error("ignoring CompletedProcedureCleaner exception: " + e.getMessage(), e);
+        }
+        proc.setStartTime(EnvironmentEdgeManager.currentTime());
+        waitingTimeout.add(proc);
+        continue;
+      }
+
+      // The procedure received an "abort-timeout", call abort() and
+      // add the procedure back in the queue for rollback.
+      if (proc.setTimeoutFailure()) {
+        long rootProcId = Procedure.getRootProcedureId(procedures, proc);
+        RootProcedureState procStack = rollbackStack.get(rootProcId);
+        procStack.abort();
+        store.update(proc);
+        runnables.addFront(proc);
+        continue;
+      }
+    }
+  }
+
+  /**
+   * Execute the rollback of the full procedure stack.
+   * Once the procedure is rolledback, the root-procedure will be visible as
+   * finished to user, and the result will be the fatal exception.
+   */
+  private boolean executeRollback(final long rootProcId, final RootProcedureState procStack) {
+    Procedure rootProc = procedures.get(rootProcId);
+    RemoteProcedureException exception = rootProc.getException();
+    if (exception == null) {
+      exception = procStack.getException();
+      rootProc.setFailure(exception);
+      store.update(rootProc);
+    }
+
+    List<Procedure> subprocStack = procStack.getSubprocedures();
+    assert subprocStack != null : "called rollback with no steps executed rootProc=" + rootProc;
+
+    int stackTail = subprocStack.size();
+    boolean reuseLock = false;
+    while (stackTail --> 0) {
+      final Procedure proc = subprocStack.get(stackTail);
+
+      if (!reuseLock && !proc.acquireLock(getEnvironment())) {
+        // can't take a lock on the procedure, add the root-proc back on the
+        // queue waiting for the lock availability
+        return false;
+      }
+
+      boolean abortRollback = !executeRollback(proc);
+      abortRollback |= !isRunning() || !store.isRunning();
+
+      // If the next procedure is the same to this one
+      // (e.g. StateMachineProcedure reuse the same instance)
+      // we can avoid to lock/unlock each step
+      reuseLock = stackTail > 0 && (subprocStack.get(stackTail - 1) == proc) && !abortRollback;
+      if (!reuseLock) {
+        proc.releaseLock(getEnvironment());
+      }
+
+      // allows to kill the executor before something is stored to the wal.
+      // useful to test the procedure recovery.
+      if (abortRollback) {
+        return false;
+      }
+
+      subprocStack.remove(stackTail);
+    }
+
+    // Finalize the procedure state
+    LOG.info("Rolledback procedure " + rootProc +
+             " exec-time=" + StringUtils.humanTimeDiff(rootProc.elapsedTime()) +
+             " exception=" + exception.getMessage());
+    procedureFinished(rootProc);
+    return true;
+  }
+
+  /**
+   * Execute the rollback of the procedure step.
+   * It updates the store with the new state (stack index)
+   * or will remove completly the procedure in case it is a child.
+   */
+  private boolean executeRollback(final Procedure proc) {
+    try {
+      proc.doRollback(getEnvironment());
+    } catch (IOException e) {
+      LOG.debug("rollback attempt failed for " + proc, e);
+      return false;
+    } catch (Throwable e) {
+      // Catch NullPointerExceptions or similar errors...
+      LOG.fatal("CODE-BUG: uncatched runtime exception for procedure: " + proc, e);
+    }
+
+    // allows to kill the executor before something is stored to the wal.
+    // useful to test the procedure recovery.
+    if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
+      LOG.debug("TESTING: Kill before store update");
+      stop();
+      return false;
+    }
+
+    if (proc.removeStackIndex()) {
+      proc.setState(ProcedureState.ROLLEDBACK);
+      if (proc.hasParent()) {
+        store.delete(proc.getProcId());
+        procedures.remove(proc.getProcId());
+      } else {
+        store.update(proc);
+      }
+    } else {
+      store.update(proc);
+    }
+    return true;
+  }
+
+  /**
+   * Executes the specified procedure
+   *  - calls the doExecute() of the procedure
+   *  - if the procedure execution didn't fail (e.g. invalid user input)
+   *     - ...and returned subprocedures
+   *        - the subprocedures are initialized.
+   *        - the subprocedures are added to the store
+   *        - the subprocedures are added to the runnable queue
+   *        - the procedure is now in a WAITING state, waiting for the subprocedures to complete
+   *     - ...if there are no subprocedure
+   *        - the procedure completed successfully
+   *        - if there is a parent (WAITING)
+   *            - the parent state will be set to RUNNABLE
+   *  - in case of failure
+   *    - the store is updated with the new state
+   *    - the executor (caller of this method) will start the rollback of the procedure
+   */
+  private void execProcedure(final RootProcedureState procStack, final Procedure procedure) {
+    Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE);
+
+    // Execute the procedure
+    boolean reExecute = false;
+    Procedure[] subprocs = null;
+    do {
+      reExecute = false;
+      try {
+        subprocs = procedure.doExecute(getEnvironment());
+        if (subprocs != null && subprocs.length == 0) {
+          subprocs = null;
+        }
+      } catch (ProcedureYieldException e) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("yield procedure: " + procedure);
+        }
+        runnables.yield(procedure);
+        return;
+      } catch (Throwable e) {
+        // Catch NullPointerExceptions or similar errors...
+        String msg = "CODE-BUG: uncatched runtime exception for procedure: " + procedure;
+        LOG.error(msg, e);
+        procedure.setFailure(new RemoteProcedureException(msg, e));
+      }
+
+      if (!procedure.isFailed()) {
+        if (subprocs != null) {
+          if (subprocs.length == 1 && subprocs[0] == procedure) {
+            // quick-shortcut for a state machine like procedure
+            subprocs = null;
+            reExecute = true;
+          } else {
+            // yield the current procedure, and make the subprocedure runnable
+            for (int i = 0; i < subprocs.length; ++i) {
+              Procedure subproc = subprocs[i];
+              if (subproc == null) {
+                String msg = "subproc[" + i + "] is null, aborting the procedure";
+                procedure.setFailure(new RemoteProcedureException(msg,
+                  new IllegalArgumentException(msg)));
+                subprocs = null;
+                break;
+              }
+
+              assert subproc.getState() == ProcedureState.INITIALIZING;
+              subproc.setParentProcId(procedure.getProcId());
+              subproc.setProcId(nextProcId());
+            }
+
+            if (!procedure.isFailed()) {
+              procedure.setChildrenLatch(subprocs.length);
+              switch (procedure.getState()) {
+                case RUNNABLE:
+                  procedure.setState(ProcedureState.WAITING);
+                  break;
+                case WAITING_TIMEOUT:
+                  waitingTimeout.add(procedure);
+                  break;
+                default:
+                  break;
+              }
+            }
+          }
+        } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) {
+          waitingTimeout.add(procedure);
+        } else {
+          // No subtask, so we are done
+          procedure.setState(ProcedureState.FINISHED);
+        }
+      }
+
+      // Add the procedure to the stack
+      procStack.addRollbackStep(procedure);
+
+      // allows to kill the executor before something is stored to the wal.
+      // useful to test the procedure recovery.
+      if (testing != null && testing.shouldKillBeforeStoreUpdate()) {
+        LOG.debug("TESTING: Kill before store update");
+        stop();
+        return;
+      }
+
+      // Commit the transaction
+      if (subprocs != null && !procedure.isFailed()) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("store add " + procedure + " children " + Arrays.toString(subprocs));
+        }
+        store.insert(procedure, subprocs);
+      } else {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("store update " + procedure);
+        }
+        store.update(procedure);
+      }
+
+      // if the store is not running we are aborting
+      if (!store.isRunning()) {
+        return;
+      }
+
+      assert (reExecute && subprocs == null) || !reExecute;
+    } while (reExecute);
+
+    // Submit the new subprocedures
+    if (subprocs != null && !procedure.isFailed()) {
+      for (int i = 0; i < subprocs.length; ++i) {
+        Procedure subproc = subprocs[i];
+        assert !procedures.containsKey(subproc.getProcId());
+        procedures.put(subproc.getProcId(), subproc);
+        runnables.addFront(subproc);
+      }
+    }
+
+    if (procedure.isFinished() && procedure.hasParent()) {
+      Procedure parent = procedures.get(procedure.getParentProcId());
+      if (parent == null) {
+        assert procStack.isRollingback();
+        return;
+      }
+
+      // If this procedure is the last child awake the parent procedure
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(parent + " child is done: " + procedure);
+      }
+      if (parent.childrenCountDown() && parent.getState() == ProcedureState.WAITING) {
+        parent.setState(ProcedureState.RUNNABLE);
+        store.update(parent);
+        runnables.addFront(parent);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace(parent + " all the children finished their work, resume.");
+        }
+        return;
+      }
+    }
+  }
+
+  private void sendProcedureLoadedNotification(final long procId) {
+    if (!this.listeners.isEmpty()) {
+      for (ProcedureExecutorListener listener: this.listeners) {
+        try {
+          listener.procedureLoaded(procId);
+        } catch (Throwable e) {
+          LOG.error("the listener " + listener + " had an error: " + e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  private void sendProcedureAddedNotification(final long procId) {
+    if (!this.listeners.isEmpty()) {
+      for (ProcedureExecutorListener listener: this.listeners) {
+        try {
+          listener.procedureAdded(procId);
+        } catch (Throwable e) {
+          LOG.error("the listener " + listener + " had an error: " + e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  private void sendProcedureFinishedNotification(final long procId) {
+    if (!this.listeners.isEmpty()) {
+      for (ProcedureExecutorListener listener: this.listeners) {
+        try {
+          listener.procedureFinished(procId);
+        } catch (Throwable e) {
+          LOG.error("the listener " + listener + " had an error: " + e.getMessage(), e);
+        }
+      }
+    }
+  }
+
+  private long nextProcId() {
+    long procId = lastProcId.incrementAndGet();
+    if (procId < 0) {
+      while (!lastProcId.compareAndSet(procId, 0)) {
+        procId = lastProcId.get();
+        if (procId >= 0)
+          break;
+      }
+      while (procedures.containsKey(procId)) {
+        procId = lastProcId.incrementAndGet();
+      }
+    }
+    return procId;
+  }
+
+  private Long getRootProcedureId(Procedure proc) {
+    return Procedure.getRootProcedureId(procedures, proc);
+  }
+
+  private void procedureFinished(final Procedure proc) {
+    // call the procedure completion cleanup handler
+    try {
+      proc.completionCleanup(getEnvironment());
+    } catch (Throwable e) {
+      // Catch NullPointerExceptions or similar errors...
+      LOG.error("CODE-BUG: uncatched runtime exception for procedure: " + proc, e);
+    }
+
+    // update the executor internal state maps
+    completed.put(proc.getProcId(), newResultFromProcedure(proc));
+    rollbackStack.remove(proc.getProcId());
+    procedures.remove(proc.getProcId());
+
+    // call the runnableSet completion cleanup handler
+    try {
+      runnables.completionCleanup(proc);
+    } catch (Throwable e) {
+      // Catch NullPointerExceptions or similar errors...
+      LOG.error("CODE-BUG: uncatched runtime exception for runnableSet: " + runnables, e);
+    }
+
+    // Notify the listeners
+    sendProcedureFinishedNotification(proc.getProcId());
+  }
+
+  public Pair<ProcedureResult, Procedure> getResultOrProcedure(final long procId) {
+    ProcedureResult result = completed.get(procId);
+    Procedure proc = null;
+    if (result == null) {
+      proc = procedures.get(procId);
+      if (proc == null) {
+        result = completed.get(procId);
+      }
+    }
+    return new Pair(result, proc);
+  }
+
+  private static ProcedureResult newResultFromProcedure(final Procedure proc) {
+    if (proc.isFailed()) {
+      return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getException());
+    }
+    return new ProcedureResult(proc.getStartTime(), proc.getLastUpdate(), proc.getResult());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
new file mode 100644
index 0000000..d8252b1
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureFairRunQueues.java
@@ -0,0 +1,172 @@
+/**
+ * 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.procedure2;
+
+import java.util.Map;
+
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * This class is a container of queues that allows to select a queue
+ * in a round robin fashion, considering priority of the queue.
+ *
+ * the quantum is just how many poll() will return the same object.
+ * e.g. if quantum is 1 and you have A and B as object you'll get: A B A B
+ * e.g. if quantum is 2 and you have A and B as object you'll get: A A B B A A B B
+ * then the object priority is just a priority * quantum
+ *
+ * Example:
+ *  - three queues (A, B, C) with priorities (1, 1, 2)
+ *  - The first poll() will return A
+ *  - The second poll() will return B
+ *  - The third and forth poll() will return C
+ *  - and so on again and again.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ProcedureFairRunQueues<TKey, TQueue extends ProcedureFairRunQueues.FairObject> {
+  private ConcurrentSkipListMap<TKey, TQueue> objMap =
+    new ConcurrentSkipListMap<TKey, TQueue>();
+
+  private final ReentrantLock lock = new ReentrantLock();
+  private final int quantum;
+
+  private Map.Entry<TKey, TQueue> current = null;
+  private int currentQuantum = 0;
+
+  public interface FairObject {
+    boolean isAvailable();
+    int getPriority();
+  }
+
+  /**
+   * @param quantum how many poll() will return the same object.
+   */
+  public ProcedureFairRunQueues(final int quantum) {
+    this.quantum = quantum;
+  }
+
+  public TQueue get(final TKey key) {
+    return objMap.get(key);
+  }
+
+  public TQueue add(final TKey key, final TQueue queue) {
+    TQueue oldq = objMap.putIfAbsent(key, queue);
+    return oldq != null ? oldq : queue;
+  }
+
+  public TQueue remove(final TKey key) {
+    TQueue queue = objMap.remove(key);
+    if (queue != null) {
+      lock.lock();
+      try {
+        if (current != null && queue == current.getValue()) {
+          currentQuantum = 0;
+          current = null;
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+    return queue;
+  }
+
+  public void clear() {
+    lock.lock();
+    try {
+      current = null;
+      objMap.clear();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * @return the next available item if present
+   */
+  public TQueue poll() {
+    lock.lock();
+    try {
+      TQueue queue;
+      if (currentQuantum == 0) {
+        if (nextObject() == null) {
+          // nothing here
+          return null;
+        }
+
+        queue = current.getValue();
+        currentQuantum = calculateQuantum(queue) - 1;
+      } else {
+        currentQuantum--;
+        queue = current.getValue();
+      }
+
+      if (!queue.isAvailable()) {
+        Map.Entry<TKey, TQueue> last = current;
+        // Try the next one
+        do {
+          if (nextObject() == null)
+            return null;
+        } while (current.getValue() != last.getValue() && !current.getValue().isAvailable());
+
+        queue = current.getValue();
+        currentQuantum = calculateQuantum(queue) - 1;
+      }
+
+      return queue;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append('{');
+    for (Map.Entry<TKey, TQueue> entry: objMap.entrySet()) {
+      builder.append(entry.getKey());
+      builder.append(':');
+      builder.append(entry.getValue());
+    }
+    builder.append('}');
+    return builder.toString();
+  }
+
+  private Map.Entry<TKey, TQueue> nextObject() {
+    Map.Entry<TKey, TQueue> next = null;
+
+    // If we have already a key, try the next one
+    if (current != null) {
+      next = objMap.higherEntry(current.getKey());
+    }
+
+    // if there is no higher key, go back to the first
+    current = (next != null) ? next : objMap.firstEntry();
+    return current;
+  }
+
+  private int calculateQuantum(final TQueue fairObject) {
+    // TODO
+    return Math.max(1, fairObject.getPriority() * quantum);
+  }
+}
\ No newline at end of file


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

Branch: refs/heads/hbase-12439
Commit: 43f3351353f7e1504f28519597181de4001354d5
Parents: 67a785d
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:53:51 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:06 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/43f33513/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/43f33513/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/43f33513/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/43f33513/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/43f33513/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/43f33513/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/43f33513/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/28] 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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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 {}
+  }
+}


[07/28] 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/9d684ac3/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>

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

Branch: refs/heads/hbase-12439
Commit: b9c4b8f6e4d94ab3335897a1caaddcaa616e2194
Parents: 7af6930
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:06:30 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:05 2015 +0100

----------------------------------------------------------------------
 .../generated/MasterProcedureProtos.java        | 1587 ++++++++++++++++--
 .../src/main/protobuf/MasterProcedure.proto     |   17 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   32 +-
 .../hadoop/hbase/master/MasterFileSystem.java   |   10 +-
 .../procedure/MasterDDLOperationHelper.java     |  167 ++
 .../master/procedure/ModifyTableProcedure.java  |  512 ++++++
 .../procedure/TestModifyTableProcedure.java     |  403 +++++
 7 files changed, 2539 insertions(+), 189 deletions(-)
----------------------------------------------------------------------



[08/28] 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/9d684ac3
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9d684ac3
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9d684ac3

Branch: refs/heads/hbase-12439
Commit: 9d684ac31f65c71db54104676cdd9458c2f881e0
Parents: aaf7751
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 20:47:46 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:05 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/9d684ac3/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/9d684ac3/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/9d684ac3/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>


[04/28] hbase git commit: HBASE-13381 Expand TestSizeFailures to include small scans (Josh Elser)

Posted by mb...@apache.org.
HBASE-13381 Expand TestSizeFailures to include small scans (Josh Elser)


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

Branch: refs/heads/hbase-12439
Commit: 3cd929eea253afd632b6def7c24df434872a2d7d
Parents: 66f7bf4
Author: tedyu <yu...@gmail.com>
Authored: Thu Apr 9 17:57:57 2015 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Thu Apr 9 17:57:57 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/client/TestSizeFailures.java   | 204 ++++++++++---------
 1 file changed, 109 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3cd929ee/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
index a77c50c..a0b0f70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
@@ -20,12 +20,10 @@ package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
 
-import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.TreeSet;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -41,16 +39,17 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.common.collect.Maps;
+
 @Category(LargeTests.class)
 public class TestSizeFailures {
-  final Log LOG = LogFactory.getLog(getClass());
+  static final Log LOG = LogFactory.getLog(TestSizeFailures.class);
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   protected static int SLAVES = 1;
+  private static TableName TABLENAME;
+  private static final int NUM_ROWS = 1000 * 1000, NUM_COLS = 10;
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // Uncomment the following lines if more verbosity is needed for
@@ -61,11 +60,49 @@ public class TestSizeFailures {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.table.sanity.checks", true); // ignore sanity checks in the server
     TEST_UTIL.startMiniCluster(SLAVES);
+
+    // Write a bunch of data
+    TABLENAME = TableName.valueOf("testSizeFailures");
+    List<byte[]> qualifiers = new ArrayList<>();
+    for (int i = 1; i <= 10; i++) {
+      qualifiers.add(Bytes.toBytes(Integer.toString(i)));
+    }
+
+    HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
+    HTableDescriptor desc = new HTableDescriptor(TABLENAME);
+    desc.addFamily(hcd);
+    byte[][] splits = new byte[9][2];
+    for (int i = 1; i < 10; i++) {
+      int split = 48 + i;
+      splits[i - 1][0] = (byte) (split >>> 8);
+      splits[i - 1][0] = (byte) (split);
+    }
+    TEST_UTIL.getHBaseAdmin().createTable(desc, splits);
+    Connection conn = TEST_UTIL.getConnection();
+
+    try (Table table = conn.getTable(TABLENAME)) {
+      List<Put> puts = new LinkedList<>();
+      for (int i = 0; i < NUM_ROWS; i++) {
+        Put p = new Put(Bytes.toBytes(Integer.toString(i)));
+        for (int j = 0; j < NUM_COLS; j++) {
+          byte[] value = new byte[50];
+          Bytes.random(value);
+          p.addColumn(FAMILY, Bytes.toBytes(Integer.toString(j)), value);
+        }
+        puts.add(p);
+
+        if (puts.size() == 1000) {
+          table.batch(puts, new Object[1000]);
+          puts.clear();
+        }
+      }
+
+      if (puts.size() > 0) {
+        table.batch(puts, new Object[puts.size()]);
+      }
+    }
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
@@ -74,89 +111,66 @@ public class TestSizeFailures {
   /**
    * Basic client side validation of HBASE-13262
    */
-   @Test
-   public void testScannerSeesAllRecords() throws Exception {
-     final int NUM_ROWS = 1000 * 1000, NUM_COLS = 10;
-     final TableName TABLENAME = TableName.valueOf("testScannerSeesAllRecords");
-     List<byte[]> qualifiers = new ArrayList<>();
-     for (int i = 1; i <= 10; i++) {
-       qualifiers.add(Bytes.toBytes(Integer.toString(i)));
-     }
-
-     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-     HTableDescriptor desc = new HTableDescriptor(TABLENAME);
-     desc.addFamily(hcd);
-     byte[][] splits = new byte[9][2];
-     for (int i = 1; i < 10; i++) {
-       int split = 48 + i;
-       splits[i - 1][0] = (byte) (split >>> 8);
-       splits[i - 1][0] = (byte) (split);
-     }
-     TEST_UTIL.getHBaseAdmin().createTable(desc, splits);
-     Connection conn = TEST_UTIL.getConnection();
-
-     try (Table table = conn.getTable(TABLENAME)) {
-       List<Put> puts = new LinkedList<>();
-       for (int i = 0; i < NUM_ROWS; i++) {
-         Put p = new Put(Bytes.toBytes(Integer.toString(i)));
-         for (int j = 0; j < NUM_COLS; j++) {
-           byte[] value = new byte[50];
-           Bytes.random(value);
-           p.addColumn(FAMILY, Bytes.toBytes(Integer.toString(j)), value);
-         }
-         puts.add(p);
-
-         if (puts.size() == 1000) {
-           Object[] results = new Object[1000];
-           try {
-             table.batch(puts, results);
-           } catch (IOException e) {
-             LOG.error("Failed to write data", e);
-             LOG.debug("Errors: " +  Arrays.toString(results));
-           }
-
-           puts.clear();
-         }
-       }
-
-       if (puts.size() > 0) {
-         Object[] results = new Object[puts.size()];
-         try {
-           table.batch(puts, results);
-         } catch (IOException e) {
-           LOG.error("Failed to write data", e);
-           LOG.debug("Errors: " +  Arrays.toString(results));
-         }
-       }
-
-       // Flush the memstore to disk
-       TEST_UTIL.getHBaseAdmin().flush(TABLENAME);
-
-       TreeSet<Integer> rows = new TreeSet<>();
-       long rowsObserved = 0l;
-       long entriesObserved = 0l;
-       Scan s = new Scan();
-       s.addFamily(FAMILY);
-       s.setMaxResultSize(-1);
-       s.setBatch(-1);
-       s.setCaching(500);
-       ResultScanner scanner = table.getScanner(s);
-       // Read all the records in the table
-       for (Result result : scanner) {
-         rowsObserved++;
-         String row = new String(result.getRow());
-         rows.add(Integer.parseInt(row));
-         while (result.advance()) {
-           entriesObserved++;
-           // result.current();
-         }
-       }
-
-       // Verify that we see 1M rows and 10M cells
-       assertEquals(NUM_ROWS, rowsObserved);
-       assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
-     }
-
-     conn.close();
-   }
+  @Test
+  public void testScannerSeesAllRecords() throws Exception {
+    Connection conn = TEST_UTIL.getConnection();
+    try (Table table = conn.getTable(TABLENAME)) {
+      Scan s = new Scan();
+      s.addFamily(FAMILY);
+      s.setMaxResultSize(-1);
+      s.setBatch(-1);
+      s.setCaching(500);
+      Entry<Long,Long> entry = sumTable(table.getScanner(s));
+      long rowsObserved = entry.getKey();
+      long entriesObserved = entry.getValue();
+
+      // Verify that we see 1M rows and 10M cells
+      assertEquals(NUM_ROWS, rowsObserved);
+      assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
+    }
+  }
+
+  /**
+   * Basic client side validation of HBASE-13262
+   */
+  @Test
+  public void testSmallScannerSeesAllRecords() throws Exception {
+    Connection conn = TEST_UTIL.getConnection();
+    try (Table table = conn.getTable(TABLENAME)) {
+      Scan s = new Scan();
+      s.setSmall(true);
+      s.addFamily(FAMILY);
+      s.setMaxResultSize(-1);
+      s.setBatch(-1);
+      s.setCaching(500);
+      Entry<Long,Long> entry = sumTable(table.getScanner(s));
+      long rowsObserved = entry.getKey();
+      long entriesObserved = entry.getValue();
+
+      // Verify that we see 1M rows and 10M cells
+      assertEquals(NUM_ROWS, rowsObserved);
+      assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
+    }
+  }
+
+  /**
+   * Count the number of rows and the number of entries from a scanner
+   *
+   * @param scanner
+   *          The Scanner
+   * @return An entry where the first item is rows observed and the second is entries observed.
+   */
+  private Entry<Long,Long> sumTable(ResultScanner scanner) {
+    long rowsObserved = 0l;
+    long entriesObserved = 0l;
+
+    // Read all the records in the table
+    for (Result result : scanner) {
+      rowsObserved++;
+      while (result.advance()) {
+        entriesObserved++;
+      }
+    }
+    return Maps.immutableEntry(rowsObserved,entriesObserved);
+  }
 }


[25/28] 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/67a785d9/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 a07516d..a9ad0e0 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -135,3 +135,33 @@ message DeleteColumnFamilyStateData {
   required bytes columnfamily_name = 3;
   optional TableSchema unmodified_table_schema = 4;
 }
+
+enum EnableTableState {
+  ENABLE_TABLE_PREPARE = 1;
+  ENABLE_TABLE_PRE_OPERATION = 2;
+  ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
+  ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
+  ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
+  ENABLE_TABLE_POST_OPERATION = 6;
+}
+
+message EnableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}
+
+enum DisableTableState {
+  DISABLE_TABLE_PREPARE = 1;
+  DISABLE_TABLE_PRE_OPERATION = 2;
+  DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
+  DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
+  DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
+  DISABLE_TABLE_POST_OPERATION = 6;
+}
+
+message DisableTableStateData {
+  required UserInformation user_info = 1;
+  required TableName table_name = 2;
+  required bool skip_table_state_check = 3;
+}

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/67a785d9/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/67a785d9/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/67a785d9/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/67a785d9/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/67a785d9/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,


[26/28] 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/67a785d9/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 4713a0a..d83ee19 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 {
 
@@ -7620,120 +7856,1901 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:DeleteColumnFamilyStateData)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_CreateTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_CreateTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_AddColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_AddColumnFamilyStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyColumnFamilyStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteColumnFamilyStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteColumnFamilyStateData_fieldAccessorTable;
+  public interface EnableTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required bool skip_table_state_check = 3;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean hasSkipTableStateCheck();
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean getSkipTableStateCheck();
   }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
-      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
-      "_info\030\001 \002(\0132\020.UserInformation\022\"\n\014table_s" +
-      "chema\030\002 \002(\0132\014.TableSchema\022 \n\013region_info" +
-      "\030\003 \003(\0132\013.RegionInfo\"\277\001\n\024ModifyTableState" +
-      "Data\022#\n\tuser_info\030\001 \002(\0132\020.UserInformatio" +
-      "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
-      "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
-      "\014.TableSchema\022&\n\036delete_column_family_in" +
-      "_modify\030\004 \002(\010\"}\n\024DeleteTableStateData\022#\n",
-      "\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\nta" +
-      "ble_name\030\002 \002(\0132\n.TableName\022 \n\013region_inf" +
-      "o\030\003 \003(\0132\013.RegionInfo\"\300\001\n\030AddColumnFamily" +
-      "StateData\022#\n\tuser_info\030\001 \002(\0132\020.UserInfor" +
-      "mation\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\0220" +
-      "\n\023columnfamily_schema\030\003 \002(\0132\023.ColumnFami" +
-      "lySchema\022-\n\027unmodified_table_schema\030\004 \001(" +
-      "\0132\014.TableSchema\"\303\001\n\033ModifyColumnFamilySt" +
-      "ateData\022#\n\tuser_info\030\001 \002(\0132\020.UserInforma" +
-      "tion\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\0220\n\023",
-      "columnfamily_schema\030\003 \002(\0132\023.ColumnFamily" +
-      "Schema\022-\n\027unmodified_table_schema\030\004 \001(\0132" +
-      "\014.TableSchema\"\254\001\n\033DeleteColumnFamilyStat" +
-      "eData\022#\n\tuser_info\030\001 \002(\0132\020.UserInformati" +
-      "on\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022\031\n\021co" +
-      "lumnfamily_name\030\003 \002(\014\022-\n\027unmodified_tabl" +
-      "e_schema\030\004 \001(\0132\014.TableSchema*\330\001\n\020CreateT" +
-      "ableState\022\036\n\032CREATE_TABLE_PRE_OPERATION\020" +
-      "\001\022 \n\034CREATE_TABLE_WRITE_FS_LAYOUT\020\002\022\034\n\030C" +
-      "REATE_TABLE_ADD_TO_META\020\003\022\037\n\033CREATE_TABL",
-      "E_ASSIGN_REGIONS\020\004\022\"\n\036CREATE_TABLE_UPDAT" +
-      "E_DESC_CACHE\020\005\022\037\n\033CREATE_TABLE_POST_OPER" +
-      "ATION\020\006*\207\002\n\020ModifyTableState\022\030\n\024MODIFY_T" +
-      "ABLE_PREPARE\020\001\022\036\n\032MODIFY_TABLE_PRE_OPERA" +
-      "TION\020\002\022(\n$MODIFY_TABLE_UPDATE_TABLE_DESC" +
-      "RIPTOR\020\003\022&\n\"MODIFY_TABLE_REMOVE_REPLICA_" +
-      "COLUMN\020\004\022!\n\035MODIFY_TABLE_DELETE_FS_LAYOU" +
-      "T\020\005\022\037\n\033MODIFY_TABLE_POST_OPERATION\020\006\022#\n\037" +
-      "MODIFY_TABLE_REOPEN_ALL_REGIONS\020\007*\337\001\n\020De" +
-      "leteTableState\022\036\n\032DELETE_TABLE_PRE_OPERA",
-      "TION\020\001\022!\n\035DELETE_TABLE_REMOVE_FROM_META\020" +
-      "\002\022 \n\034DELETE_TABLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036D" +
-      "ELETE_TABLE_UPDATE_DESC_CACHE\020\004\022!\n\035DELET" +
-      "E_TABLE_UNASSIGN_REGIONS\020\005\022\037\n\033DELETE_TAB" +
-      "LE_POST_OPERATION\020\006*\331\001\n\024AddColumnFamilyS" +
-      "tate\022\035\n\031ADD_COLUMN_FAMILY_PREPARE\020\001\022#\n\037A" +
-      "DD_COLUMN_FAMILY_PRE_OPERATION\020\002\022-\n)ADD_" +
-      "COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022" +
-      "$\n ADD_COLUMN_FAMILY_POST_OPERATION\020\004\022(\n" +
-      "$ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS\020\005*",
-      "\353\001\n\027ModifyColumnFamilyState\022 \n\034MODIFY_CO" +
-      "LUMN_FAMILY_PREPARE\020\001\022&\n\"MODIFY_COLUMN_F" +
-      "AMILY_PRE_OPERATION\020\002\0220\n,MODIFY_COLUMN_F" +
-      "AMILY_UPDATE_TABLE_DESCRIPTOR\020\003\022\'\n#MODIF" +
-      "Y_COLUMN_FAMILY_POST_OPERATION\020\004\022+\n\'MODI" +
-      "FY_COLUMN_FAMILY_REOPEN_ALL_REGIONS\020\005*\226\002" +
-      "\n\027DeleteColumnFamilyState\022 \n\034DELETE_COLU" +
-      "MN_FAMILY_PREPARE\020\001\022&\n\"DELETE_COLUMN_FAM" +
-      "ILY_PRE_OPERATION\020\002\0220\n,DELETE_COLUMN_FAM" +
-      "ILY_UPDATE_TABLE_DESCRIPTOR\020\003\022)\n%DELETE_",
-      "COLUMN_FAMILY_DELETE_FS_LAYOUT\020\004\022\'\n#DELE" +
-      "TE_COLUMN_FAMILY_POST_OPERATION\020\005\022+\n\'DEL" +
-      "ETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS\020\006BK" +
-      "\n*org.apache.hadoop.hbase.protobuf.gener" +
-      "atedB\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 =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_CreateTableStateData_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+  /**
+   * Protobuf type {@code EnableTableStateData}
+   */
+  public static final class EnableTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements EnableTableStateDataOrBuilder {
+    // Use EnableTableStateData.newBuilder() to construct.
+    private EnableTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private EnableTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final EnableTableStateData defaultInstance;
+    public static EnableTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public EnableTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private EnableTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              skipTableStateCheck_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<EnableTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<EnableTableStateData>() {
+      public EnableTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new EnableTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<EnableTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required bool skip_table_state_check = 3;
+    public static final int SKIP_TABLE_STATE_CHECK_FIELD_NUMBER = 3;
+    private boolean skipTableStateCheck_;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean hasSkipTableStateCheck() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean getSkipTableStateCheck() {
+      return skipTableStateCheck_;
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      skipTableStateCheck_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSkipTableStateCheck()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, skipTableStateCheck_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, skipTableStateCheck_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasSkipTableStateCheck() == other.hasSkipTableStateCheck());
+      if (hasSkipTableStateCheck()) {
+        result = result && (getSkipTableStateCheck()
+            == other.getSkipTableStateCheck());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasSkipTableStateCheck()) {
+        hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipTableStateCheck());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code EnableTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        skipTableStateCheck_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_EnableTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.skipTableStateCheck_ = skipTableStateCheck_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasSkipTableStateCheck()) {
+          setSkipTableStateCheck(other.getSkipTableStateCheck());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasSkipTableStateCheck()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required bool skip_table_state_check = 3;
+      private boolean skipTableStateCheck_ ;
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean hasSkipTableStateCheck() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean getSkipTableStateCheck() {
+        return skipTableStateCheck_;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder setSkipTableStateCheck(boolean value) {
+        bitField0_ |= 0x00000004;
+        skipTableStateCheck_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder clearSkipTableStateCheck() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        skipTableStateCheck_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:EnableTableStateData)
+    }
+
+    static {
+      defaultInstance = new EnableTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:EnableTableStateData)
+  }
+
+  public interface DisableTableStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required bool skip_table_state_check = 3;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean hasSkipTableStateCheck();
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    boolean getSkipTableStateCheck();
+  }
+  /**
+   * Protobuf type {@code DisableTableStateData}
+   */
+  public static final class DisableTableStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements DisableTableStateDataOrBuilder {
+    // Use DisableTableStateData.newBuilder() to construct.
+    private DisableTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private DisableTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final DisableTableStateData defaultInstance;
+    public static DisableTableStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public DisableTableStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private DisableTableStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              skipTableStateCheck_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<DisableTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<DisableTableStateData>() {
+      public DisableTableStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new DisableTableStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<DisableTableStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required bool skip_table_state_check = 3;
+    public static final int SKIP_TABLE_STATE_CHECK_FIELD_NUMBER = 3;
+    private boolean skipTableStateCheck_;
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean hasSkipTableStateCheck() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required bool skip_table_state_check = 3;</code>
+     */
+    public boolean getSkipTableStateCheck() {
+      return skipTableStateCheck_;
+    }
+
+    private void initFields() {
+      userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      skipTableStateCheck_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasUserInfo()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSkipTableStateCheck()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getUserInfo().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(3, skipTableStateCheck_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, userInfo_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, tableName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(3, skipTableStateCheck_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasSkipTableStateCheck() == other.hasSkipTableStateCheck());
+      if (hasSkipTableStateCheck()) {
+        result = result && (getSkipTableStateCheck()
+            == other.getSkipTableStateCheck());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasUserInfo()) {
+        hash = (37 * hash) + USER_INFO_FIELD_NUMBER;
+        hash = (53 * hash) + getUserInfo().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasSkipTableStateCheck()) {
+        hash = (37 * hash) + SKIP_TABLE_STATE_CHECK_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getSkipTableStateCheck());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code DisableTableStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        skipTableStateCheck_ = false;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DisableTableStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.skipTableStateCheck_ = skipTableStateCheck_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasSkipTableStateCheck()) {
+          setSkipTableStateCheck(other.getSkipTableStateCheck());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasSkipTableStateCheck()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required bool skip_table_state_check = 3;
+      private boolean skipTableStateCheck_ ;
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean hasSkipTableStateCheck() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public boolean getSkipTableStateCheck() {
+        return skipTableStateCheck_;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder setSkipTableStateCheck(boolean value) {
+        bitField0_ |= 0x00000004;
+        skipTableStateCheck_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool skip_table_state_check = 3;</code>
+       */
+      public Builder clearSkipTableStateCheck() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        skipTableStateCheck_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:DisableTableStateData)
+    }
+
+    static {
+      defaultInstance = new DisableTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:DisableTableStateData)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CreateTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CreateTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_AddColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_AddColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DeleteColumnFamilyStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DeleteColumnFamilyStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EnableTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EnableTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_DisableTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_DisableTableStateData_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
+      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
+      "_info\030\001

<TRUNCATED>

[06/28] 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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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/9d684ac3/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;
+      }
+    }
+  }
+}


[02/28] hbase git commit: HBASE-13275 Setting hbase.security.authorization to false does not disable authorization

Posted by mb...@apache.org.
HBASE-13275 Setting hbase.security.authorization to false does not disable authorization


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

Branch: refs/heads/hbase-12439
Commit: ed703762aeebfce9855858834a3fa6a4f51a8977
Parents: 1deadb6
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Apr 9 13:50:35 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Apr 9 14:47:21 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/security/User.java  |    2 +
 .../hbase/security/access/AccessController.java |  308 +++--
 .../visibility/VisibilityController.java        |  100 +-
 .../hbase/security/HBaseKerberosUtils.java      |    8 +-
 .../hbase/security/access/SecureTestUtil.java   |   78 +-
 .../security/access/TestAccessController.java   |   89 +-
 .../access/TestWithDisabledAuthorization.java   | 1079 ++++++++++++++++++
 .../TestWithDisabledAuthorization.java          |  237 ++++
 8 files changed, 1679 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
index a5ac51a..58a3c66 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/security/User.java
@@ -52,6 +52,8 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 public abstract class User {
   public static final String HBASE_SECURITY_CONF_KEY =
       "hbase.security.authentication";
+  public static final String HBASE_SECURITY_AUTHORIZATION_CONF_KEY =
+      "hbase.security.authorization";
 
   protected UserGroupInformation ugi;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index cd8f5ff..7b306c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -165,11 +165,11 @@ public class AccessController extends BaseMasterAndRegionObserver
 
   TableAuthManager authManager = null;
 
-  // flags if we are running on a region of the _acl_ table
+  /** flags if we are running on a region of the _acl_ table */
   boolean aclRegion = false;
 
-  // defined only for Endpoint implementation, so it can have way to
-  // access region services.
+  /** defined only for Endpoint implementation, so it can have way to
+   access region services */
   private RegionCoprocessorEnvironment regionEnv;
 
   /** Mapping of scanner instances to the user who created them */
@@ -178,25 +178,30 @@ public class AccessController extends BaseMasterAndRegionObserver
 
   private Map<TableName, List<UserPermission>> tableAcls;
 
-  // Provider for mapping principal names to Users
+  /** Provider for mapping principal names to Users */
   private UserProvider userProvider;
 
-  // The list of users with superuser authority
+  /** The list of users with superuser authority */
   private List<String> superusers;
 
-  // if we are able to support cell ACLs
+  /** if we are active, usually true, only not true if "hbase.security.authorization"
+   has been set to false in site configuration */
+  boolean authorizationEnabled;
+
+  /** if we are able to support cell ACLs */
   boolean cellFeaturesEnabled;
 
-  // if we should check EXEC permissions
+  /** if we should check EXEC permissions */
   boolean shouldCheckExecPermission;
 
-  // if we should terminate access checks early as soon as table or CF grants
-  // allow access; pre-0.98 compatible behavior
+  /** if we should terminate access checks early as soon as table or CF grants
+    allow access; pre-0.98 compatible behavior */
   boolean compatibleEarlyTermination;
 
+  /** if we have been successfully initialized */
   private volatile boolean initialized = false;
 
-  // This boolean having relevance only in the Master.
+  /** if the ACL table is available, only relevant in the master */
   private volatile boolean aclTabAvailable = false;
 
   public Region getRegion() {
@@ -405,8 +410,8 @@ public class AccessController extends BaseMasterAndRegionObserver
    * @throws IOException if obtaining the current user fails
    * @throws AccessDeniedException if user has no authorization
    */
-  private void requirePermission(String request, TableName tableName, byte[] family, byte[] qualifier,
-      Action... permissions) throws IOException {
+  private void requirePermission(String request, TableName tableName, byte[] family,
+      byte[] qualifier, Action... permissions) throws IOException {
     User user = getActiveUser();
     AuthResult result = null;
 
@@ -422,7 +427,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
     logResult(result);
-    if (!result.isAllowed()) {
+    if (authorizationEnabled && !result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -455,7 +460,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
     logResult(result);
-    if (!result.isAllowed()) {
+    if (authorizationEnabled && !result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -485,7 +490,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
     logResult(result);
-    if (!result.isAllowed()) {
+    if (authorizationEnabled && !result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
     }
   }
@@ -501,31 +506,6 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   /**
-   * Authorizes that the current user has permission to perform the given
-   * action on the set of table column families.
-   * @param perm Action that is required
-   * @param env The current coprocessor environment
-   * @param families The map of column families-qualifiers.
-   * @throws AccessDeniedException if the authorization check failed
-   */
-  private void requirePermission(String request, Action perm,
-        RegionCoprocessorEnvironment env,
-        Map<byte[], ? extends Collection<?>> families)
-      throws IOException {
-    User user = getActiveUser();
-    AuthResult result = permissionGranted(request, user, perm, env, families);
-    logResult(result);
-
-    if (!result.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions (table=" +
-        env.getRegion().getTableDesc().getTableName()+
-        ((families != null && families.size() > 0) ? ", family: " +
-        result.toFamilyString() : "") + ", action=" +
-        perm.toString() + ")");
-    }
-  }
-
-  /**
    * Checks that the user has the given global permission. The generated
    * audit log message will contain context information for the operation
    * being authorized, based on the given parameters.
@@ -545,9 +525,11 @@ public class AccessController extends BaseMasterAndRegionObserver
       result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
       result.getParams().setTableName(tableName).setFamilies(familyMap);
       logResult(result);
-      throw new AccessDeniedException("Insufficient permissions for user '" +
+      if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions for user '" +
           (user != null ? user.getShortName() : "null") +"' (global, action=" +
           perm.toString() + ")");
+      }
     }
   }
 
@@ -570,9 +552,11 @@ public class AccessController extends BaseMasterAndRegionObserver
       authResult = AuthResult.deny(request, "Global check failed", user, perm, null);
       authResult.getParams().setNamespace(namespace);
       logResult(authResult);
-      throw new AccessDeniedException("Insufficient permissions for user '" +
+      if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions for user '" +
           (user != null ? user.getShortName() : "null") +"' (global, action=" +
           perm.toString() + ")");
+      }
     }
   }
 
@@ -598,7 +582,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
     logResult(result);
-    if (!result.isAllowed()) {
+    if (authorizationEnabled && !result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions "
           + result.toContextString());
     }
@@ -629,7 +613,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       }
     }
     logResult(result);
-    if (!result.isAllowed()) {
+    if (authorizationEnabled && !result.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions "
           + result.toContextString());
     }
@@ -764,6 +748,8 @@ public class AccessController extends BaseMasterAndRegionObserver
             }
           }
         }
+      } else if (entry.getValue() == null) {
+        get.addFamily(col);        
       } else {
         throw new RuntimeException("Unhandled collection type " +
           entry.getValue().getClass().getName());
@@ -899,8 +885,14 @@ public class AccessController extends BaseMasterAndRegionObserver
   // Checks whether incoming cells contain any tag with type as ACL_TAG_TYPE. This tag
   // type is reserved and should not be explicitly set by user.
   private void checkForReservedTagPresence(User user, Mutation m) throws IOException {
+    // No need to check if we're not going to throw
+    if (!authorizationEnabled) {
+      m.setAttribute(TAG_CHECK_PASSED, TRUE);
+      return;
+    }
     // Superusers are allowed to store cells unconditionally.
     if (superusers.contains(user.getShortName())) {
+      m.setAttribute(TAG_CHECK_PASSED, TRUE);
       return;
     }
     // We already checked (prePut vs preBatchMutation)
@@ -928,6 +920,11 @@ public class AccessController extends BaseMasterAndRegionObserver
     CompoundConfiguration conf = new CompoundConfiguration();
     conf.add(env.getConfiguration());
 
+    authorizationEnabled = conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+    if (!authorizationEnabled) {
+      LOG.warn("The AccessController has been loaded with authorization checks disabled.");
+    }
+
     shouldCheckExecPermission = conf.getBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY,
       AccessControlConstants.DEFAULT_EXEC_PERMISSION_CHECKS);
 
@@ -1064,6 +1061,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c,
       final TableName tableName) throws IOException {
     requirePermission("truncateTable", tableName, null, null, Action.ADMIN, Action.CREATE);
+
     final Configuration conf = c.getEnvironment().getConfiguration();
     User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
       @Override
@@ -1163,8 +1161,12 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
       throws IOException {
     if (Bytes.equals(tableName.getName(), AccessControlLists.ACL_GLOBAL_NAME)) {
+      // We have to unconditionally disallow disable of the ACL table when we are installed,
+      // even if not enforcing authorizations. We are still allowing grants and revocations,
+      // checking permissions and logging audit messages, etc. If the ACL table is not
+      // available we will fail random actions all over the place.
       throw new AccessDeniedException("Not allowed to disable "
-          + AccessControlLists.ACL_TABLE_NAME + " table.");
+          + AccessControlLists.ACL_TABLE_NAME + " table with AccessController installed");
     }
     requirePermission("disableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
   }
@@ -1243,6 +1245,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       final SnapshotDescription snapshot) throws IOException {
     if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
       // list it, if user is the owner of snapshot
+      // TODO: We are not logging this for audit
     } else {
       requirePermission("listSnapshot", Action.ADMIN);
     }
@@ -1272,6 +1275,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       final SnapshotDescription snapshot) throws IOException {
     if (SnapshotDescriptionUtils.isSnapshotOwner(snapshot, getActiveUser())) {
       // Snapshot owner is allowed to delete the snapshot
+      // TODO: We are not logging this for audit
     } else {
       requirePermission("deleteSnapshot", Action.ADMIN);
     }
@@ -1439,8 +1443,9 @@ public class AccessController extends BaseMasterAndRegionObserver
       authResult.setReason("Covering cell set");
     }
     logResult(authResult);
-    if (!authResult.isAllowed()) {
-      throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+    if (authorizationEnabled && !authResult.isAllowed()) {
+      throw new AccessDeniedException("Insufficient permissions " +
+        authResult.toContextString());
     }
   }
 
@@ -1483,26 +1488,29 @@ public class AccessController extends BaseMasterAndRegionObserver
         // grants three times (permissionGranted above, here, and in the
         // filter) but that's the price of backwards compatibility.
         if (hasFamilyQualifierPermission(user, Action.READ, env, families)) {
-          Filter ourFilter = new AccessControlFilter(authManager, user, table,
-            AccessControlFilter.Strategy.CHECK_TABLE_AND_CF_ONLY,
-            cfVsMaxVersions);
-          // wrap any existing filter
-          if (filter != null) {
-            ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
-              Lists.newArrayList(ourFilter, filter));
-          }
           authResult.setAllowed(true);
           authResult.setReason("Access allowed with filter");
-          switch (opType) {
-          case GET:
-          case EXISTS:
-            ((Get)query).setFilter(ourFilter);
-            break;
-          case SCAN:
-            ((Scan)query).setFilter(ourFilter);
-            break;
-          default:
-            throw new RuntimeException("Unhandled operation " + opType);
+          // Only wrap the filter if we are enforcing authorizations
+          if (authorizationEnabled) {
+            Filter ourFilter = new AccessControlFilter(authManager, user, table,
+              AccessControlFilter.Strategy.CHECK_TABLE_AND_CF_ONLY,
+              cfVsMaxVersions);
+            // wrap any existing filter
+            if (filter != null) {
+              ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
+                Lists.newArrayList(ourFilter, filter));
+            }
+            switch (opType) {
+              case GET:
+              case EXISTS:
+                ((Get)query).setFilter(ourFilter);
+                break;
+              case SCAN:
+                ((Scan)query).setFilter(ourFilter);
+                break;
+              default:
+                throw new RuntimeException("Unhandled operation " + opType);
+            }
           }
         }
       } else {
@@ -1510,31 +1518,34 @@ public class AccessController extends BaseMasterAndRegionObserver
         // than whole table or CF. Simply inject a filter and return what is
         // allowed. We will not throw an AccessDeniedException. This is a
         // behavioral change since 0.96.
-        Filter ourFilter = new AccessControlFilter(authManager, user, table,
-          AccessControlFilter.Strategy.CHECK_CELL_DEFAULT, cfVsMaxVersions);
-        // wrap any existing filter
-        if (filter != null) {
-          ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
-            Lists.newArrayList(ourFilter, filter));
-        }
         authResult.setAllowed(true);
         authResult.setReason("Access allowed with filter");
-        switch (opType) {
-        case GET:
-        case EXISTS:
-          ((Get)query).setFilter(ourFilter);
-          break;
-        case SCAN:
-          ((Scan)query).setFilter(ourFilter);
-          break;
-        default:
-          throw new RuntimeException("Unhandled operation " + opType);
+        // Only wrap the filter if we are enforcing authorizations
+        if (authorizationEnabled) {
+          Filter ourFilter = new AccessControlFilter(authManager, user, table,
+            AccessControlFilter.Strategy.CHECK_CELL_DEFAULT, cfVsMaxVersions);
+          // wrap any existing filter
+          if (filter != null) {
+            ourFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL,
+              Lists.newArrayList(ourFilter, filter));
+          }
+          switch (opType) {
+            case GET:
+            case EXISTS:
+              ((Get)query).setFilter(ourFilter);
+              break;
+            case SCAN:
+              ((Scan)query).setFilter(ourFilter);
+              break;
+            default:
+              throw new RuntimeException("Unhandled operation " + opType);
+          }
         }
       }
     }
 
     logResult(authResult);
-    if (!authResult.isAllowed()) {
+    if (authorizationEnabled && !authResult.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions (table=" + table +
         ", action=READ)");
     }
@@ -1557,14 +1568,15 @@ public class AccessController extends BaseMasterAndRegionObserver
   public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Put put, final WALEdit edit, final Durability durability)
       throws IOException {
+    User user = getActiveUser();
+    checkForReservedTagPresence(user, put);
+
     // Require WRITE permission to the table, CF, or top visible value, if any.
     // NOTE: We don't need to check the permissions for any earlier Puts
     // because we treat the ACLs in each Put as timestamped like any other
     // HBase value. A new ACL in a new Put applies to that Put. It doesn't
     // change the ACL of any previous Put. This allows simple evolution of
     // security policy over time without requiring expensive updates.
-    User user = getActiveUser();
-    checkForReservedTagPresence(user, put);
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<Cell>> families = put.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.PUT, user, env, families, Action.WRITE);
@@ -1572,10 +1584,11 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         put.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
+      } else if (authorizationEnabled) {
         throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
       }
     }
+
     // Add cell ACLs from the operation to the cells themselves
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
@@ -1616,8 +1629,9 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         delete.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      } else if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
   }
@@ -1640,18 +1654,18 @@ public class AccessController extends BaseMasterAndRegionObserver
             opType = OpType.DELETE;
           }
           AuthResult authResult = null;
-          if (checkCoveringPermission(opType, c.getEnvironment(), m.getRow(), m.getFamilyCellMap(),
-              m.getTimeStamp(), Action.WRITE)) {
-            authResult = AuthResult.allow(opType.toString(), "Covering cell set", getActiveUser(),
-                Action.WRITE, table, m.getFamilyCellMap());
+          if (checkCoveringPermission(opType, c.getEnvironment(), m.getRow(),
+            m.getFamilyCellMap(), m.getTimeStamp(), Action.WRITE)) {
+            authResult = AuthResult.allow(opType.toString(), "Covering cell set",
+              getActiveUser(), Action.WRITE, table, m.getFamilyCellMap());
           } else {
-            authResult = AuthResult.deny(opType.toString(), "Covering cell set", getActiveUser(),
-                Action.WRITE, table, m.getFamilyCellMap());
+            authResult = AuthResult.deny(opType.toString(), "Covering cell set",
+              getActiveUser(), Action.WRITE, table, m.getFamilyCellMap());
           }
           logResult(authResult);
-          if (!authResult.isAllowed()) {
+          if (authorizationEnabled && !authResult.isAllowed()) {
             throw new AccessDeniedException("Insufficient permissions "
-                + authResult.toContextString());
+              + authResult.toContextString());
           }
         }
       }
@@ -1673,9 +1687,10 @@ public class AccessController extends BaseMasterAndRegionObserver
       final CompareFilter.CompareOp compareOp,
       final ByteArrayComparable comparator, final Put put,
       final boolean result) throws IOException {
-    // Require READ and WRITE permissions on the table, CF, and KV to update
     User user = getActiveUser();
     checkForReservedTagPresence(user, put);
+
+    // Require READ and WRITE permissions on the table, CF, and KV to update
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<byte[]>> families = makeFamilyMap(family, qualifier);
     AuthResult authResult = permissionGranted(OpType.CHECK_AND_PUT, user, env, families,
@@ -1684,10 +1699,12 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         put.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      } else if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
+
     byte[] bytes = put.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
       if (cellFeaturesEnabled) {
@@ -1719,7 +1736,7 @@ public class AccessController extends BaseMasterAndRegionObserver
             getActiveUser(), Action.READ, table, families);
       }
       logResult(authResult);
-      if (!authResult.isAllowed()) {
+      if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
       }
     }
@@ -1748,8 +1765,9 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         delete.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      } else if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
     return result;
@@ -1776,7 +1794,7 @@ public class AccessController extends BaseMasterAndRegionObserver
             getActiveUser(), Action.READ, table, families);
       }
       logResult(authResult);
-      if (!authResult.isAllowed()) {
+      if (authorizationEnabled && !authResult.isAllowed()) {
         throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
       }
     }
@@ -1801,7 +1819,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       authResult.setReason("Covering cell set");
     }
     logResult(authResult);
-    if (!authResult.isAllowed()) {
+    if (authorizationEnabled && !authResult.isAllowed()) {
       throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
     }
     return -1;
@@ -1810,9 +1828,10 @@ public class AccessController extends BaseMasterAndRegionObserver
   @Override
   public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> c, Append append)
       throws IOException {
-    // Require WRITE permission to the table, CF, and the KV to be appended
     User user = getActiveUser();
     checkForReservedTagPresence(user, append);
+
+    // Require WRITE permission to the table, CF, and the KV to be appended
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<Cell>> families = append.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.APPEND, user, env, families, Action.WRITE);
@@ -1820,10 +1839,12 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         append.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      } else if (authorizationEnabled)  {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
+
     byte[] bytes = append.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
       if (cellFeaturesEnabled) {
@@ -1832,6 +1853,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
       }
     }
+
     return null;
   }
 
@@ -1852,8 +1874,9 @@ public class AccessController extends BaseMasterAndRegionObserver
             getActiveUser(), Action.WRITE, table, append.getFamilyCellMap());
       }
       logResult(authResult);
-      if (!authResult.isAllowed()) {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      if (authorizationEnabled && !authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
     return null;
@@ -1863,10 +1886,11 @@ public class AccessController extends BaseMasterAndRegionObserver
   public Result preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Increment increment)
       throws IOException {
-    // Require WRITE permission to the table, CF, and the KV to be replaced by
-    // the incremented value
     User user = getActiveUser();
     checkForReservedTagPresence(user, increment);
+
+    // Require WRITE permission to the table, CF, and the KV to be replaced by
+    // the incremented value
     RegionCoprocessorEnvironment env = c.getEnvironment();
     Map<byte[],? extends Collection<Cell>> families = increment.getFamilyCellMap();
     AuthResult authResult = permissionGranted(OpType.INCREMENT, user, env, families,
@@ -1875,10 +1899,12 @@ public class AccessController extends BaseMasterAndRegionObserver
     if (!authResult.isAllowed()) {
       if (cellFeaturesEnabled && !compatibleEarlyTermination) {
         increment.setAttribute(CHECK_COVERING_PERM, TRUE);
-      } else {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      } else if (authorizationEnabled) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
+
     byte[] bytes = increment.getAttribute(AccessControlConstants.OP_ATTRIBUTE_ACL);
     if (bytes != null) {
       if (cellFeaturesEnabled) {
@@ -1887,6 +1913,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         throw new DoNotRetryIOException("Cell ACLs cannot be persisted");
       }
     }
+
     return null;
   }
 
@@ -1907,8 +1934,9 @@ public class AccessController extends BaseMasterAndRegionObserver
             getActiveUser(), Action.WRITE, table, increment.getFamilyCellMap());
       }
       logResult(authResult);
-      if (!authResult.isAllowed()) {
-        throw new AccessDeniedException("Insufficient permissions " + authResult.toContextString());
+      if (authorizationEnabled && !authResult.isAllowed()) {
+        throw new AccessDeniedException("Insufficient permissions " +
+          authResult.toContextString());
       }
     }
     return null;
@@ -1990,7 +2018,8 @@ public class AccessController extends BaseMasterAndRegionObserver
   public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
       final Scan scan, final RegionScanner s) throws IOException {
     User user = getActiveUser();
-    if (user != null && user.getShortName() != null) {      // store reference to scanner owner for later checks
+    if (user != null && user.getShortName() != null) {
+      // store reference to scanner owner for later checks
       scannerOwners.put(s, user.getShortName());
     }
     return s;
@@ -2025,7 +2054,7 @@ public class AccessController extends BaseMasterAndRegionObserver
   private void requireScannerOwner(InternalScanner s) throws AccessDeniedException {
     String requestUserName = RpcServer.getRequestUserName();
     String owner = scannerOwners.get(s);
-    if (owner != null && !owner.equals(requestUserName)) {
+    if (authorizationEnabled && owner != null && !owner.equals(requestUserName)) {
       throw new AccessDeniedException("User '"+ requestUserName +"' is not the scanner owner!");
     }
   }
@@ -2119,11 +2148,11 @@ public class AccessController extends BaseMasterAndRegionObserver
           case Global :
           case Table :
             requirePermission("grant", perm.getTableName(), perm.getFamily(),
-                perm.getQualifier(), Action.ADMIN);
+              perm.getQualifier(), Action.ADMIN);
             break;
           case Namespace :
             requireGlobalPermission("grant", Action.ADMIN, perm.getNamespace());
-            break;
+           break;
         }
 
         User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
@@ -2170,7 +2199,7 @@ public class AccessController extends BaseMasterAndRegionObserver
           case Global :
           case Table :
             requirePermission("revoke", perm.getTableName(), perm.getFamily(),
-                              perm.getQualifier(), Action.ADMIN);
+              perm.getQualifier(), Action.ADMIN);
             break;
           case Namespace :
             requireGlobalPermission("revoke", Action.ADMIN, perm.getNamespace());
@@ -2264,9 +2293,12 @@ public class AccessController extends BaseMasterAndRegionObserver
     }
     AccessControlProtos.CheckPermissionsResponse response = null;
     try {
+      User user = getActiveUser();
       TableName tableName = regionEnv.getRegion().getTableDesc().getTableName();
       for (Permission permission : permissions) {
         if (permission instanceof TablePermission) {
+          // Check table permissions
+
           TablePermission tperm = (TablePermission) permission;
           for (Action action : permission.getActions()) {
             if (!tperm.getTableName().equals(tableName)) {
@@ -2276,7 +2308,8 @@ public class AccessController extends BaseMasterAndRegionObserver
                   tperm.getTableName()));
             }
 
-            Map<byte[], Set<byte[]>> familyMap = new TreeMap<byte[], Set<byte[]>>(Bytes.BYTES_COMPARATOR);
+            Map<byte[], Set<byte[]>> familyMap =
+                new TreeMap<byte[], Set<byte[]>>(Bytes.BYTES_COMPARATOR);
             if (tperm.getFamily() != null) {
               if (tperm.getQualifier() != null) {
                 Set<byte[]> qualifiers = Sets.newTreeSet(Bytes.BYTES_COMPARATOR);
@@ -2287,12 +2320,37 @@ public class AccessController extends BaseMasterAndRegionObserver
               }
             }
 
-            requirePermission("checkPermissions", action, regionEnv, familyMap);
+            AuthResult result = permissionGranted("checkPermissions", user, action, regionEnv,
+              familyMap);
+            logResult(result);
+            if (!result.isAllowed()) {
+              // Even if passive we need to throw an exception here, we support checking
+              // effective permissions, so throw unconditionally
+              throw new AccessDeniedException("Insufficient permissions (table=" + tableName +
+                (familyMap.size() > 0 ? ", family: " + result.toFamilyString() : "") +
+                ", action=" + action.toString() + ")");
+            }
           }
 
         } else {
+          // Check global permissions
+
           for (Action action : permission.getActions()) {
-            requirePermission("checkPermissions", action);
+            AuthResult result;
+            if (authManager.authorize(user, action)) {
+              result = AuthResult.allow("checkPermissions", "Global action allowed", user,
+                action, null, null);
+            } else {
+              result = AuthResult.deny("checkPermissions", "Global action denied", user, action,
+                null, null);
+            }
+            logResult(result);
+            if (!result.isAllowed()) {
+              // Even if passive we need to throw an exception here, we support checking
+              // effective permissions, so throw unconditionally
+              throw new AccessDeniedException("Insufficient permissions (action=" +
+                action.toString() + ")");
+            }
           }
         }
       }
@@ -2335,6 +2393,10 @@ public class AccessController extends BaseMasterAndRegionObserver
   }
 
   private void isSystemOrSuperUser(Configuration conf) throws IOException {
+    // No need to check if we're not going to throw
+    if (!authorizationEnabled) {
+      return;
+    }
     User user = userProvider.getCurrent();
     if (user == null) {
       throw new IOException("Unable to obtain the current user, " +

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 5fb0234..6e659ef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -125,7 +125,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   // flags if we are running on a region of the 'labels' table
   private boolean labelsRegion = false;
   // Flag denoting whether AcessController is available or not.
-  private boolean acOn = false;
+  private boolean accessControllerAvailable = false;
   private Configuration conf;
   private volatile boolean initialized = false;
   private boolean checkAuths = false;
@@ -137,6 +137,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   private List<String> superGroups;
   private VisibilityLabelService visibilityLabelService;
 
+  /** if we are active, usually true, only not true if "hbase.security.authorization"
+    has been set to false in site configuration */
+  boolean authorizationEnabled;
+
   // Add to this list if there are any reserved tag types
   private static ArrayList<Byte> RESERVED_VIS_TAG_TYPES = new ArrayList<Byte>();
   static {
@@ -148,6 +152,12 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void start(CoprocessorEnvironment env) throws IOException {
     this.conf = env.getConfiguration();
+
+    authorizationEnabled = conf.getBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+    if (!authorizationEnabled) {
+      LOG.warn("The VisibilityController has been loaded with authorization checks disabled.");
+    }
+
     if (HFile.getFormatVersion(conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
       throw new RuntimeException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
         + " is required to persist visibility labels. Consider setting " + HFile.FORMAT_VERSION_KEY
@@ -200,6 +210,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
       TableName tableName, HTableDescriptor htd) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     if (LABELS_TABLE_NAME.equals(tableName)) {
       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
     }
@@ -208,6 +221,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HColumnDescriptor column) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     if (LABELS_TABLE_NAME.equals(tableName)) {
       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
     }
@@ -216,6 +232,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
       TableName tableName, HColumnDescriptor descriptor) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     if (LABELS_TABLE_NAME.equals(tableName)) {
       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
     }
@@ -224,6 +243,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
       TableName tableName, byte[] c) throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     if (LABELS_TABLE_NAME.equals(tableName)) {
       throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
     }
@@ -232,6 +254,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
+    if (!authorizationEnabled) {
+      return;
+    }
     if (LABELS_TABLE_NAME.equals(tableName)) {
       throw new ConstraintException("Cannot disable " + LABELS_TABLE_NAME);
     }
@@ -244,7 +269,8 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     // Read the entire labels table and populate the zk
     if (e.getEnvironment().getRegion().getRegionInfo().getTable().equals(LABELS_TABLE_NAME)) {
       this.labelsRegion = true;
-      this.acOn = CoprocessorHost.getLoadedCoprocessors().contains(AccessController.class.getName());
+      this.accessControllerAvailable = CoprocessorHost.getLoadedCoprocessors()
+          .contains(AccessController.class.getName());
       // Defer the init of VisibilityLabelService on labels region until it is in recovering state.
       if (!e.getEnvironment().getRegion().isRecovering()) {
         initVisibilityLabelService(e.getEnvironment());
@@ -298,9 +324,12 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
         pair = checkForReservedVisibilityTagPresence(cellScanner.current(), pair);
         if (!pair.getFirst()) {
-          miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE,
+          // Don't disallow reserved tags if authorization is disabled
+          if (authorizationEnabled) {
+            miniBatchOp.setOperationStatus(i, new OperationStatus(SANITY_CHECK_FAILURE,
               "Mutation contains cell with reserved type tag"));
-          sanityFailure = true;
+            sanityFailure = true;
+          }
           break;
         } else {
           // Indicates that the cell has a the tag which was modified in the src replication cluster
@@ -319,7 +348,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
           List<Tag> visibilityTags = labelCache.get(labelsExp);
           if (visibilityTags == null) {
             // Don't check user auths for labels with Mutations when the user is super user
-            boolean authCheck = this.checkAuths && !(isSystemOrSuperUser());
+            boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser());
             try {
               visibilityTags = this.visibilityLabelService.createVisibilityExpTags(labelsExp, true,
                   authCheck);
@@ -366,6 +395,11 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   public void prePrepareTimeStampForDeleteVersion(
       ObserverContext<RegionCoprocessorEnvironment> ctx, Mutation delete, Cell cell,
       byte[] byteNow, Get get) throws IOException {
+    // Nothing to do if we are not filtering by visibility
+    if (!authorizationEnabled) {
+      return;
+    }
+
     CellVisibility cellVisibility = null;
     try {
       cellVisibility = delete.getCellVisibility();
@@ -513,6 +547,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     if (!initialized) {
       throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
     }
+    // Nothing to do if authorization is not enabled
+    if (!authorizationEnabled) {
+      return s;
+    }
     Region region = e.getEnvironment().getRegion();
     Authorizations authorizations = null;
     try {
@@ -547,6 +585,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   public DeleteTracker postInstantiateDeleteTracker(
       ObserverContext<RegionCoprocessorEnvironment> ctx, DeleteTracker delTracker)
       throws IOException {
+    // Nothing to do if we are not filtering by visibility
+    if (!authorizationEnabled) {
+      return delTracker;
+    }
     Region region = ctx.getEnvironment().getRegion();
     TableName table = region.getRegionInfo().getTable();
     if (table.isSystemTable()) {
@@ -599,16 +641,20 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     // This is duplicated code!
     String requestUName = RpcServer.getRequestUserName();
     String owner = scannerOwners.get(s);
-    if (owner != null && !owner.equals(requestUName)) {
+    if (authorizationEnabled && owner != null && !owner.equals(requestUName)) {
       throw new AccessDeniedException("User '" + requestUName + "' is not the scanner owner!");
     }
   }
 
   @Override
-  public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
-      throws IOException {
+  public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get,
+      List<Cell> results) throws IOException {
     if (!initialized) {
-      throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized!");
+      throw new VisibilityControllerNotReadyException("VisibilityController not yet initialized");
+    }
+    // Nothing useful to do if authorization is not enabled
+    if (!authorizationEnabled) {
+      return;
     }
     Region region = e.getEnvironment().getRegion();
     Authorizations authorizations = null;
@@ -657,6 +703,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public Result preAppend(ObserverContext<RegionCoprocessorEnvironment> e, Append append)
       throws IOException {
+    // If authorization is not enabled, we don't care about reserved tags
+    if (!authorizationEnabled) {
+      return null;
+    }
     for (CellScanner cellScanner = append.cellScanner(); cellScanner.advance();) {
       if (!checkForReservedVisibilityTagPresence(cellScanner.current())) {
         throw new FailedSanityCheckException("Append contains cell with reserved type tag");
@@ -668,6 +718,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   @Override
   public Result preIncrement(ObserverContext<RegionCoprocessorEnvironment> e, Increment increment)
       throws IOException {
+    // If authorization is not enabled, we don't care about reserved tags
+    if (!authorizationEnabled) {
+      return null;
+    }
     for (CellScanner cellScanner = increment.cellScanner(); cellScanner.advance();) {
       if (!checkForReservedVisibilityTagPresence(cellScanner.current())) {
         throw new FailedSanityCheckException("Increment contains cell with reserved type tag");
@@ -691,7 +745,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     }
     // Prepend new visibility tags to a new list of tags for the cell
     // Don't check user auths for labels with Mutations when the user is super user
-    boolean authCheck = this.checkAuths && !(isSystemOrSuperUser());
+    boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser());
     tags.addAll(this.visibilityLabelService.createVisibilityExpTags(cellVisibility.getExpression(),
         true, authCheck));
     // Save an object allocation where we can
@@ -730,7 +784,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     } else {
       List<byte[]> labels = new ArrayList<byte[]>(visLabels.size());
       try {
-        checkCallingUserAuth();
+        if (authorizationEnabled) {
+          checkCallingUserAuth();
+        }
         RegionActionResult successResult = RegionActionResult.newBuilder().build();
         for (VisibilityLabel visLabel : visLabels) {
           byte[] label = visLabel.getLabel().toByteArray();
@@ -790,8 +846,9 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       byte[] user = request.getUser().toByteArray();
       List<byte[]> labelAuths = new ArrayList<byte[]>(auths.size());
       try {
-        checkCallingUserAuth();
-
+        if (authorizationEnabled) {
+          checkCallingUserAuth();
+        }
         for (ByteString authBS : auths) {
           labelAuths.add(authBS.toByteArray());
         }
@@ -862,7 +919,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       try {
         // We do ACL check here as we create scanner directly on region. It will not make calls to
         // AccessController CP methods.
-        if (this.acOn && !isSystemOrSuperUser()) {
+        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
           User requestingUser = VisibilityUtils.getActiveUser();
           throw new AccessDeniedException("User '"
               + (requestingUser != null ? requestingUser.getShortName() : "null")
@@ -905,13 +962,15 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       List<byte[]> labelAuths = new ArrayList<byte[]>(auths.size());
       try {
         // When AC is ON, do AC based user auth check
-        if (this.acOn && !isSystemOrSuperUser()) {
+        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
           User user = VisibilityUtils.getActiveUser();
           throw new AccessDeniedException("User '" + (user != null ? user.getShortName() : "null")
               + " is not authorized to perform this action.");
         }
-        checkCallingUserAuth(); // When AC is not in place the calling user should have SYSTEM_LABEL
-                                // auth to do this action.
+        if (authorizationEnabled) {
+          checkCallingUserAuth(); // When AC is not in place the calling user should have
+                                  // SYSTEM_LABEL auth to do this action.
+        }
         for (ByteString authBS : auths) {
           labelAuths.add(authBS.toByteArray());
         }
@@ -955,7 +1014,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       try {
         // We do ACL check here as we create scanner directly on region. It will not make calls to
         // AccessController CP methods.
-        if (this.acOn && !isSystemOrSuperUser()) {
+        if (authorizationEnabled && accessControllerAvailable && !isSystemOrSuperUser()) {
           User requestingUser = VisibilityUtils.getActiveUser();
           throw new AccessDeniedException("User '"
               + (requestingUser != null ? requestingUser.getShortName() : "null")
@@ -979,7 +1038,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
   }
 
   private void checkCallingUserAuth() throws IOException {
-    if (!this.acOn) {
+    if (!authorizationEnabled) { // Redundant, but just in case
+      return;
+    }
+    if (!accessControllerAvailable) {
       User user = VisibilityUtils.getActiveUser();
       if (user == null) {
         throw new IOException("Unable to retrieve calling user");

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java
index 1f9f4f5..237efe9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/HBaseKerberosUtils.java
@@ -62,8 +62,8 @@ public class HBaseKerberosUtils {
   public static Configuration getConfigurationWoPrincipal() {
     Configuration conf = HBaseConfiguration.create();
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    conf.set("hbase.security.authentication", "kerberos");
-    conf.setBoolean("hbase.security.authorization", true);
+    conf.set(User.HBASE_SECURITY_CONF_KEY, "kerberos");
+    conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
     return conf;
   }
 
@@ -75,8 +75,8 @@ public class HBaseKerberosUtils {
 
   public static void setSecuredConfiguration(Configuration conf) {
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    conf.set("hbase.security.authentication", "kerberos");
-    conf.setBoolean("hbase.security.authorization", true);
+    conf.set(User.HBASE_SECURITY_CONF_KEY, "kerberos");
+    conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
     conf.set(KRB_KEYTAB_FILE, System.getProperty(KRB_KEYTAB_FILE));
     conf.set(KRB_PRINCIPAL, System.getProperty(KRB_PRINCIPAL));
     conf.set(MASTER_KRB_PRINCIPAL, System.getProperty(KRB_PRINCIPAL));

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index b30c770..fb06c05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -55,11 +55,13 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 
 import com.google.common.collect.Lists;
@@ -75,14 +77,7 @@ public class SecureTestUtil {
   private static final Log LOG = LogFactory.getLog(SecureTestUtil.class);
   private static final int WAIT_TIME = 10000;
 
-  public static void enableSecurity(Configuration conf) throws IOException {
-    conf.set("hadoop.security.authorization", "false");
-    conf.set("hadoop.security.authentication", "simple");
-    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
-      "," + MasterSyncObserver.class.getName());
-    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
-      "," + SecureBulkLoadEndpoint.class.getName());
-    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+  public static void configureSuperuser(Configuration conf) throws IOException {
     // The secure minicluster creates separate service principals based on the
     // current user's name, one for each slave. We need to add all of these to
     // the superuser list or security won't function properly. We expect the
@@ -97,8 +92,19 @@ public class SecureTestUtil {
       sb.append(currentUser); sb.append(".hfs."); sb.append(i);
     }
     conf.set("hbase.superuser", sb.toString());
+  }
+
+  public static void enableSecurity(Configuration conf) throws IOException {
+    conf.set("hadoop.security.authorization", "false");
+    conf.set("hadoop.security.authentication", "simple");
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+      "," + MasterSyncObserver.class.getName());
+    conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+      "," + SecureBulkLoadEndpoint.class.getName());
+    conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
     // Need HFile V3 for tags for security features
     conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
+    configureSuperuser(conf);
   }
 
   public static void verifyConfiguration(Configuration conf) {
@@ -716,4 +722,60 @@ public class SecureTestUtil {
   public static String convertToGroup(String group) {
     return AccessControlLists.GROUP_PREFIX + group;
   }
+
+  public static void checkGlobalPerms(HBaseTestingUtility testUtil, Permission.Action... actions)
+      throws IOException {
+    Permission[] perms = new Permission[actions.length];
+    for (int i = 0; i < actions.length; i++) {
+      perms[i] = new Permission(actions[i]);
+    }
+    CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
+    for (Action a : actions) {
+      request.addPermission(AccessControlProtos.Permission.newBuilder()
+          .setType(AccessControlProtos.Permission.Type.Global)
+          .setGlobalPermission(
+              AccessControlProtos.GlobalPermission.newBuilder()
+                  .addAction(ProtobufUtil.toPermissionAction(a)).build()));
+    }
+    try(Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration());
+        Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
+      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(channel);
+      try {
+        protocol.checkPermissions(null, request.build());
+      } catch (ServiceException se) {
+        ProtobufUtil.toIOException(se);
+      }
+    }
+  }
+
+  public static void checkTablePerms(HBaseTestingUtility testUtil, TableName table, byte[] family,
+      byte[] column, Permission.Action... actions) throws IOException {
+    Permission[] perms = new Permission[actions.length];
+    for (int i = 0; i < actions.length; i++) {
+      perms[i] = new TablePermission(table, family, column, actions[i]);
+    }
+    checkTablePerms(testUtil, table, perms);
+  }
+
+  public static void checkTablePerms(HBaseTestingUtility testUtil, TableName table,
+      Permission... perms) throws IOException {
+    CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
+    for (Permission p : perms) {
+      request.addPermission(ProtobufUtil.toPermission(p));
+    }
+
+    try(Connection conn = ConnectionFactory.createConnection(testUtil.getConfiguration());
+        Table acl = conn.getTable(table)) {
+      AccessControlService.BlockingInterface protocol =
+        AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0]));
+      try {
+        protocol.checkPermissions(null, request.build());
+      } catch (ServiceException se) {
+        ProtobufUtil.toIOException(se);
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ed703762/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 58b8587..ff0a720 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -138,7 +138,7 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Rule public TestTableName TEST_TABLE = new TestTableName();
-  private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Configuration conf;
 
   /** The systemUserConnection created here is tied to the system user. In case, you are planning
@@ -1537,60 +1537,6 @@ public class TestAccessController extends SecureTestUtil {
     verifyDenied(action, USER_CREATE, USER_RW, USER_NONE, USER_RO);
   }
 
-  public void checkGlobalPerms(Permission.Action... actions) throws IOException {
-    Permission[] perms = new Permission[actions.length];
-    for (int i = 0; i < actions.length; i++) {
-      perms[i] = new Permission(actions[i]);
-    }
-    CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
-    for (Action a : actions) {
-      request.addPermission(AccessControlProtos.Permission.newBuilder()
-          .setType(AccessControlProtos.Permission.Type.Global)
-          .setGlobalPermission(
-              AccessControlProtos.GlobalPermission.newBuilder()
-                  .addAction(ProtobufUtil.toPermissionAction(a)).build()));
-    }
-    try(Connection conn = ConnectionFactory.createConnection(conf);
-        Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
-      BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(channel);
-      try {
-        protocol.checkPermissions(null, request.build());
-      } catch (ServiceException se) {
-        ProtobufUtil.toIOException(se);
-      }
-    }
-  }
-
-  public void checkTablePerms(TableName table, byte[] family, byte[] column,
-      Permission.Action... actions) throws IOException {
-    Permission[] perms = new Permission[actions.length];
-    for (int i = 0; i < actions.length; i++) {
-      perms[i] = new TablePermission(table, family, column, actions[i]);
-    }
-
-    checkTablePerms(table, perms);
-  }
-
-  public void checkTablePerms(TableName table, Permission... perms) throws IOException {
-    CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
-    for (Permission p : perms) {
-      request.addPermission(ProtobufUtil.toPermission(p));
-    }
-
-    try(Connection conn = ConnectionFactory.createConnection(conf);
-        Table acl = conn.getTable(table)) {
-      AccessControlService.BlockingInterface protocol =
-        AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0]));
-      try {
-        protocol.checkPermissions(null, request.build());
-      } catch (ServiceException se) {
-        ProtobufUtil.toIOException(se);
-      }
-    }
-  }
-
   @Test
   public void testCheckPermissions() throws Exception {
     // --------------------------------------
@@ -1598,7 +1544,7 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction globalAdmin = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkGlobalPerms(Permission.Action.ADMIN);
+        checkGlobalPerms(TEST_UTIL, Permission.Action.ADMIN);
         return null;
       }
     };
@@ -1610,7 +1556,7 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction globalReadWrite = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkGlobalPerms(Permission.Action.READ, Permission.Action.WRITE);
+        checkGlobalPerms(TEST_UTIL, Permission.Action.READ, Permission.Action.WRITE);
         return null;
       }
     };
@@ -1639,7 +1585,8 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction tableRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), null, null, Permission.Action.READ);
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), null, null,
+          Permission.Action.READ);
         return null;
       }
     };
@@ -1647,7 +1594,8 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction columnRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, null, Permission.Action.READ);
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
+          Permission.Action.READ);
         return null;
       }
     };
@@ -1655,7 +1603,8 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction qualifierRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1, Permission.Action.READ);
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+          Permission.Action.READ);
         return null;
       }
     };
@@ -1663,9 +1612,11 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction multiQualifierRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), new Permission[] {
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1, Permission.Action.READ),
-            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2, Permission.Action.READ), });
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[] {
+            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q1,
+              Permission.Action.READ),
+            new TablePermission(TEST_TABLE.getTableName(), TEST_FAMILY, TEST_Q2,
+              Permission.Action.READ), });
         return null;
       }
     };
@@ -1673,8 +1624,10 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction globalAndTableRead = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), new Permission[] { new Permission(Permission.Action.READ),
-            new TablePermission(TEST_TABLE.getTableName(), null, (byte[]) null, Permission.Action.READ), });
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(),
+          new Permission[] { new Permission(Permission.Action.READ),
+            new TablePermission(TEST_TABLE.getTableName(), null, (byte[]) null,
+            Permission.Action.READ), });
         return null;
       }
     };
@@ -1682,7 +1635,7 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction noCheck = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), new Permission[0]);
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), new Permission[0]);
         return null;
       }
     };
@@ -1708,8 +1661,8 @@ public class TestAccessController extends SecureTestUtil {
     AccessTestAction familyReadWrite = new AccessTestAction() {
       @Override
       public Void run() throws Exception {
-        checkTablePerms(TEST_TABLE.getTableName(), TEST_FAMILY, null, Permission.Action.READ,
-          Permission.Action.WRITE);
+        checkTablePerms(TEST_UTIL, TEST_TABLE.getTableName(), TEST_FAMILY, null,
+          Permission.Action.READ, Permission.Action.WRITE);
         return null;
       }
     };


[10/28] hbase git commit: HBASE-13202 Procedure v2 - core framework

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/aaf77519/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java
new file mode 100644
index 0000000..3c7dcdb
--- /dev/null
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ProcedureProtos.java
@@ -0,0 +1,7219 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: Procedure.proto
+
+package org.apache.hadoop.hbase.protobuf.generated;
+
+public final class ProcedureProtos {
+  private ProcedureProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  /**
+   * Protobuf enum {@code ProcedureState}
+   */
+  public enum ProcedureState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>INITIALIZING = 1;</code>
+     *
+     * <pre>
+     * Procedure in construction, not yet added to the executor
+     * </pre>
+     */
+    INITIALIZING(0, 1),
+    /**
+     * <code>RUNNABLE = 2;</code>
+     *
+     * <pre>
+     * Procedure added to the executor, and ready to be executed
+     * </pre>
+     */
+    RUNNABLE(1, 2),
+    /**
+     * <code>WAITING = 3;</code>
+     *
+     * <pre>
+     * The procedure is waiting on children to be completed
+     * </pre>
+     */
+    WAITING(2, 3),
+    /**
+     * <code>WAITING_TIMEOUT = 4;</code>
+     *
+     * <pre>
+     * The procedure is waiting a timout or an external event
+     * </pre>
+     */
+    WAITING_TIMEOUT(3, 4),
+    /**
+     * <code>ROLLEDBACK = 5;</code>
+     *
+     * <pre>
+     * The procedure failed and was rolledback
+     * </pre>
+     */
+    ROLLEDBACK(4, 5),
+    /**
+     * <code>FINISHED = 6;</code>
+     *
+     * <pre>
+     * The procedure execution is completed. may need a rollback if failed.
+     * </pre>
+     */
+    FINISHED(5, 6),
+    ;
+
+    /**
+     * <code>INITIALIZING = 1;</code>
+     *
+     * <pre>
+     * Procedure in construction, not yet added to the executor
+     * </pre>
+     */
+    public static final int INITIALIZING_VALUE = 1;
+    /**
+     * <code>RUNNABLE = 2;</code>
+     *
+     * <pre>
+     * Procedure added to the executor, and ready to be executed
+     * </pre>
+     */
+    public static final int RUNNABLE_VALUE = 2;
+    /**
+     * <code>WAITING = 3;</code>
+     *
+     * <pre>
+     * The procedure is waiting on children to be completed
+     * </pre>
+     */
+    public static final int WAITING_VALUE = 3;
+    /**
+     * <code>WAITING_TIMEOUT = 4;</code>
+     *
+     * <pre>
+     * The procedure is waiting a timout or an external event
+     * </pre>
+     */
+    public static final int WAITING_TIMEOUT_VALUE = 4;
+    /**
+     * <code>ROLLEDBACK = 5;</code>
+     *
+     * <pre>
+     * The procedure failed and was rolledback
+     * </pre>
+     */
+    public static final int ROLLEDBACK_VALUE = 5;
+    /**
+     * <code>FINISHED = 6;</code>
+     *
+     * <pre>
+     * The procedure execution is completed. may need a rollback if failed.
+     * </pre>
+     */
+    public static final int FINISHED_VALUE = 6;
+
+
+    public final int getNumber() { return value; }
+
+    public static ProcedureState valueOf(int value) {
+      switch (value) {
+        case 1: return INITIALIZING;
+        case 2: return RUNNABLE;
+        case 3: return WAITING;
+        case 4: return WAITING_TIMEOUT;
+        case 5: return ROLLEDBACK;
+        case 6: return FINISHED;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<ProcedureState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<ProcedureState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<ProcedureState>() {
+            public ProcedureState findValueByNumber(int number) {
+              return ProcedureState.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.ProcedureProtos.getDescriptor().getEnumTypes().get(0);
+    }
+
+    private static final ProcedureState[] VALUES = values();
+
+    public static ProcedureState 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 ProcedureState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:ProcedureState)
+  }
+
+  public interface ProcedureOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string class_name = 1;
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    boolean hasClassName();
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    java.lang.String getClassName();
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    com.google.protobuf.ByteString
+        getClassNameBytes();
+
+    // optional uint64 parent_id = 2;
+    /**
+     * <code>optional uint64 parent_id = 2;</code>
+     *
+     * <pre>
+     * parent if not a root-procedure otherwise not set
+     * </pre>
+     */
+    boolean hasParentId();
+    /**
+     * <code>optional uint64 parent_id = 2;</code>
+     *
+     * <pre>
+     * parent if not a root-procedure otherwise not set
+     * </pre>
+     */
+    long getParentId();
+
+    // required uint64 proc_id = 3;
+    /**
+     * <code>required uint64 proc_id = 3;</code>
+     */
+    boolean hasProcId();
+    /**
+     * <code>required uint64 proc_id = 3;</code>
+     */
+    long getProcId();
+
+    // required uint64 start_time = 4;
+    /**
+     * <code>required uint64 start_time = 4;</code>
+     */
+    boolean hasStartTime();
+    /**
+     * <code>required uint64 start_time = 4;</code>
+     */
+    long getStartTime();
+
+    // optional string owner = 5;
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    boolean hasOwner();
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    java.lang.String getOwner();
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getOwnerBytes();
+
+    // required .ProcedureState state = 6;
+    /**
+     * <code>required .ProcedureState state = 6;</code>
+     *
+     * <pre>
+     * internal "runtime" state
+     * </pre>
+     */
+    boolean hasState();
+    /**
+     * <code>required .ProcedureState state = 6;</code>
+     *
+     * <pre>
+     * internal "runtime" state
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState getState();
+
+    // repeated uint32 stack_id = 7;
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    java.util.List<java.lang.Integer> getStackIdList();
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    int getStackIdCount();
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    int getStackId(int index);
+
+    // required uint64 last_update = 8;
+    /**
+     * <code>required uint64 last_update = 8;</code>
+     */
+    boolean hasLastUpdate();
+    /**
+     * <code>required uint64 last_update = 8;</code>
+     */
+    long getLastUpdate();
+
+    // optional uint32 timeout = 9;
+    /**
+     * <code>optional uint32 timeout = 9;</code>
+     */
+    boolean hasTimeout();
+    /**
+     * <code>optional uint32 timeout = 9;</code>
+     */
+    int getTimeout();
+
+    // optional .ForeignExceptionMessage exception = 10;
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    boolean hasException();
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage getException();
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder getExceptionOrBuilder();
+
+    // optional bytes result = 11;
+    /**
+     * <code>optional bytes result = 11;</code>
+     *
+     * <pre>
+     * opaque (user) result structure
+     * </pre>
+     */
+    boolean hasResult();
+    /**
+     * <code>optional bytes result = 11;</code>
+     *
+     * <pre>
+     * opaque (user) result structure
+     * </pre>
+     */
+    com.google.protobuf.ByteString getResult();
+
+    // optional bytes state_data = 12;
+    /**
+     * <code>optional bytes state_data = 12;</code>
+     *
+     * <pre>
+     * opaque (user) procedure internal-state
+     * </pre>
+     */
+    boolean hasStateData();
+    /**
+     * <code>optional bytes state_data = 12;</code>
+     *
+     * <pre>
+     * opaque (user) procedure internal-state
+     * </pre>
+     */
+    com.google.protobuf.ByteString getStateData();
+  }
+  /**
+   * Protobuf type {@code Procedure}
+   *
+   * <pre>
+   **
+   * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
+   * </pre>
+   */
+  public static final class Procedure extends
+      com.google.protobuf.GeneratedMessage
+      implements ProcedureOrBuilder {
+    // Use Procedure.newBuilder() to construct.
+    private Procedure(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private Procedure(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final Procedure defaultInstance;
+    public static Procedure getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public Procedure getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private Procedure(
+        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: {
+              bitField0_ |= 0x00000001;
+              className_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              parentId_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              procId_ = input.readUInt64();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              startTime_ = input.readUInt64();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000010;
+              owner_ = input.readBytes();
+              break;
+            }
+            case 48: {
+              int rawValue = input.readEnum();
+              org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState value = org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(6, rawValue);
+              } else {
+                bitField0_ |= 0x00000020;
+                state_ = value;
+              }
+              break;
+            }
+            case 56: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                stackId_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              stackId_.add(input.readUInt32());
+              break;
+            }
+            case 58: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040) && input.getBytesUntilLimit() > 0) {
+                stackId_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                stackId_.add(input.readUInt32());
+              }
+              input.popLimit(limit);
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000040;
+              lastUpdate_ = input.readUInt64();
+              break;
+            }
+            case 72: {
+              bitField0_ |= 0x00000080;
+              timeout_ = input.readUInt32();
+              break;
+            }
+            case 82: {
+              org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000100) == 0x00000100)) {
+                subBuilder = exception_.toBuilder();
+              }
+              exception_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(exception_);
+                exception_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000100;
+              break;
+            }
+            case 90: {
+              bitField0_ |= 0x00000200;
+              result_ = input.readBytes();
+              break;
+            }
+            case 98: {
+              bitField0_ |= 0x00000400;
+              stateData_ = input.readBytes();
+              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_ & 0x00000040) == 0x00000040)) {
+          stackId_ = java.util.Collections.unmodifiableList(stackId_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_Procedure_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_Procedure_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.class, org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<Procedure> PARSER =
+        new com.google.protobuf.AbstractParser<Procedure>() {
+      public Procedure parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new Procedure(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<Procedure> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string class_name = 1;
+    public static final int CLASS_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object className_;
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    public boolean hasClassName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    public java.lang.String getClassName() {
+      java.lang.Object ref = className_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          className_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string class_name = 1;</code>
+     *
+     * <pre>
+     * internal "static" state
+     * </pre>
+     */
+    public com.google.protobuf.ByteString
+        getClassNameBytes() {
+      java.lang.Object ref = className_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        className_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // optional uint64 parent_id = 2;
+    public static final int PARENT_ID_FIELD_NUMBER = 2;
+    private long parentId_;
+    /**
+     * <code>optional uint64 parent_id = 2;</code>
+     *
+     * <pre>
+     * parent if not a root-procedure otherwise not set
+     * </pre>
+     */
+    public boolean hasParentId() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional uint64 parent_id = 2;</code>
+     *
+     * <pre>
+     * parent if not a root-procedure otherwise not set
+     * </pre>
+     */
+    public long getParentId() {
+      return parentId_;
+    }
+
+    // required uint64 proc_id = 3;
+    public static final int PROC_ID_FIELD_NUMBER = 3;
+    private long procId_;
+    /**
+     * <code>required uint64 proc_id = 3;</code>
+     */
+    public boolean hasProcId() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required uint64 proc_id = 3;</code>
+     */
+    public long getProcId() {
+      return procId_;
+    }
+
+    // required uint64 start_time = 4;
+    public static final int START_TIME_FIELD_NUMBER = 4;
+    private long startTime_;
+    /**
+     * <code>required uint64 start_time = 4;</code>
+     */
+    public boolean hasStartTime() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required uint64 start_time = 4;</code>
+     */
+    public long getStartTime() {
+      return startTime_;
+    }
+
+    // optional string owner = 5;
+    public static final int OWNER_FIELD_NUMBER = 5;
+    private java.lang.Object owner_;
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    public boolean hasOwner() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    public java.lang.String getOwner() {
+      java.lang.Object ref = owner_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          owner_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string owner = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getOwnerBytes() {
+      java.lang.Object ref = owner_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        owner_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required .ProcedureState state = 6;
+    public static final int STATE_FIELD_NUMBER = 6;
+    private org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState state_;
+    /**
+     * <code>required .ProcedureState state = 6;</code>
+     *
+     * <pre>
+     * internal "runtime" state
+     * </pre>
+     */
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>required .ProcedureState state = 6;</code>
+     *
+     * <pre>
+     * internal "runtime" state
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState getState() {
+      return state_;
+    }
+
+    // repeated uint32 stack_id = 7;
+    public static final int STACK_ID_FIELD_NUMBER = 7;
+    private java.util.List<java.lang.Integer> stackId_;
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    public java.util.List<java.lang.Integer>
+        getStackIdList() {
+      return stackId_;
+    }
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    public int getStackIdCount() {
+      return stackId_.size();
+    }
+    /**
+     * <code>repeated uint32 stack_id = 7;</code>
+     *
+     * <pre>
+     * stack indices in case the procedure was running
+     * </pre>
+     */
+    public int getStackId(int index) {
+      return stackId_.get(index);
+    }
+
+    // required uint64 last_update = 8;
+    public static final int LAST_UPDATE_FIELD_NUMBER = 8;
+    private long lastUpdate_;
+    /**
+     * <code>required uint64 last_update = 8;</code>
+     */
+    public boolean hasLastUpdate() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>required uint64 last_update = 8;</code>
+     */
+    public long getLastUpdate() {
+      return lastUpdate_;
+    }
+
+    // optional uint32 timeout = 9;
+    public static final int TIMEOUT_FIELD_NUMBER = 9;
+    private int timeout_;
+    /**
+     * <code>optional uint32 timeout = 9;</code>
+     */
+    public boolean hasTimeout() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional uint32 timeout = 9;</code>
+     */
+    public int getTimeout() {
+      return timeout_;
+    }
+
+    // optional .ForeignExceptionMessage exception = 10;
+    public static final int EXCEPTION_FIELD_NUMBER = 10;
+    private org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage exception_;
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    public boolean hasException() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage getException() {
+      return exception_;
+    }
+    /**
+     * <code>optional .ForeignExceptionMessage exception = 10;</code>
+     *
+     * <pre>
+     * user state/results
+     * </pre>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder getExceptionOrBuilder() {
+      return exception_;
+    }
+
+    // optional bytes result = 11;
+    public static final int RESULT_FIELD_NUMBER = 11;
+    private com.google.protobuf.ByteString result_;
+    /**
+     * <code>optional bytes result = 11;</code>
+     *
+     * <pre>
+     * opaque (user) result structure
+     * </pre>
+     */
+    public boolean hasResult() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>optional bytes result = 11;</code>
+     *
+     * <pre>
+     * opaque (user) result structure
+     * </pre>
+     */
+    public com.google.protobuf.ByteString getResult() {
+      return result_;
+    }
+
+    // optional bytes state_data = 12;
+    public static final int STATE_DATA_FIELD_NUMBER = 12;
+    private com.google.protobuf.ByteString stateData_;
+    /**
+     * <code>optional bytes state_data = 12;</code>
+     *
+     * <pre>
+     * opaque (user) procedure internal-state
+     * </pre>
+     */
+    public boolean hasStateData() {
+      return ((bitField0_ & 0x00000400) == 0x00000400);
+    }
+    /**
+     * <code>optional bytes state_data = 12;</code>
+     *
+     * <pre>
+     * opaque (user) procedure internal-state
+     * </pre>
+     */
+    public com.google.protobuf.ByteString getStateData() {
+      return stateData_;
+    }
+
+    private void initFields() {
+      className_ = "";
+      parentId_ = 0L;
+      procId_ = 0L;
+      startTime_ = 0L;
+      owner_ = "";
+      state_ = org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState.INITIALIZING;
+      stackId_ = java.util.Collections.emptyList();
+      lastUpdate_ = 0L;
+      timeout_ = 0;
+      exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
+      result_ = com.google.protobuf.ByteString.EMPTY;
+      stateData_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasClassName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasProcId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasStartTime()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasState()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasLastUpdate()) {
+        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.writeBytes(1, getClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeUInt64(2, parentId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt64(3, procId_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeUInt64(4, startTime_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(5, getOwnerBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeEnum(6, state_.getNumber());
+      }
+      for (int i = 0; i < stackId_.size(); i++) {
+        output.writeUInt32(7, stackId_.get(i));
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeUInt64(8, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeUInt32(9, timeout_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeMessage(10, exception_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeBytes(11, result_);
+      }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeBytes(12, stateData_);
+      }
+      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
+          .computeBytesSize(1, getClassNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(2, parentId_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(3, procId_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(4, startTime_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getOwnerBytes());
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(6, state_.getNumber());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < stackId_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeUInt32SizeNoTag(stackId_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getStackIdList().size();
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(8, lastUpdate_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt32Size(9, timeout_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(10, exception_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(11, result_);
+      }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(12, stateData_);
+      }
+      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.ProcedureProtos.Procedure)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure other = (org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure) obj;
+
+      boolean result = true;
+      result = result && (hasClassName() == other.hasClassName());
+      if (hasClassName()) {
+        result = result && getClassName()
+            .equals(other.getClassName());
+      }
+      result = result && (hasParentId() == other.hasParentId());
+      if (hasParentId()) {
+        result = result && (getParentId()
+            == other.getParentId());
+      }
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
+      result = result && (hasStartTime() == other.hasStartTime());
+      if (hasStartTime()) {
+        result = result && (getStartTime()
+            == other.getStartTime());
+      }
+      result = result && (hasOwner() == other.hasOwner());
+      if (hasOwner()) {
+        result = result && getOwner()
+            .equals(other.getOwner());
+      }
+      result = result && (hasState() == other.hasState());
+      if (hasState()) {
+        result = result &&
+            (getState() == other.getState());
+      }
+      result = result && getStackIdList()
+          .equals(other.getStackIdList());
+      result = result && (hasLastUpdate() == other.hasLastUpdate());
+      if (hasLastUpdate()) {
+        result = result && (getLastUpdate()
+            == other.getLastUpdate());
+      }
+      result = result && (hasTimeout() == other.hasTimeout());
+      if (hasTimeout()) {
+        result = result && (getTimeout()
+            == other.getTimeout());
+      }
+      result = result && (hasException() == other.hasException());
+      if (hasException()) {
+        result = result && getException()
+            .equals(other.getException());
+      }
+      result = result && (hasResult() == other.hasResult());
+      if (hasResult()) {
+        result = result && getResult()
+            .equals(other.getResult());
+      }
+      result = result && (hasStateData() == other.hasStateData());
+      if (hasStateData()) {
+        result = result && getStateData()
+            .equals(other.getStateData());
+      }
+      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 (hasClassName()) {
+        hash = (37 * hash) + CLASS_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getClassName().hashCode();
+      }
+      if (hasParentId()) {
+        hash = (37 * hash) + PARENT_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getParentId());
+      }
+      if (hasProcId()) {
+        hash = (37 * hash) + PROC_ID_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getProcId());
+      }
+      if (hasStartTime()) {
+        hash = (37 * hash) + START_TIME_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getStartTime());
+      }
+      if (hasOwner()) {
+        hash = (37 * hash) + OWNER_FIELD_NUMBER;
+        hash = (53 * hash) + getOwner().hashCode();
+      }
+      if (hasState()) {
+        hash = (37 * hash) + STATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashEnum(getState());
+      }
+      if (getStackIdCount() > 0) {
+        hash = (37 * hash) + STACK_ID_FIELD_NUMBER;
+        hash = (53 * hash) + getStackIdList().hashCode();
+      }
+      if (hasLastUpdate()) {
+        hash = (37 * hash) + LAST_UPDATE_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getLastUpdate());
+      }
+      if (hasTimeout()) {
+        hash = (37 * hash) + TIMEOUT_FIELD_NUMBER;
+        hash = (53 * hash) + getTimeout();
+      }
+      if (hasException()) {
+        hash = (37 * hash) + EXCEPTION_FIELD_NUMBER;
+        hash = (53 * hash) + getException().hashCode();
+      }
+      if (hasResult()) {
+        hash = (37 * hash) + RESULT_FIELD_NUMBER;
+        hash = (53 * hash) + getResult().hashCode();
+      }
+      if (hasStateData()) {
+        hash = (37 * hash) + STATE_DATA_FIELD_NUMBER;
+        hash = (53 * hash) + getStateData().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure 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.ProcedureProtos.Procedure parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure 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.ProcedureProtos.Procedure parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure 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.ProcedureProtos.Procedure parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure 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.ProcedureProtos.Procedure parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure 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.ProcedureProtos.Procedure 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 Procedure}
+     *
+     * <pre>
+     **
+     * Procedure metadata, serialized by the ProcedureStore to be able to recover the old state.
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_Procedure_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_Procedure_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.class, org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getExceptionFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        className_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        parentId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        startTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        owner_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        state_ = org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState.INITIALIZING;
+        bitField0_ = (bitField0_ & ~0x00000020);
+        stackId_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000040);
+        lastUpdate_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        timeout_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000100);
+        if (exceptionBuilder_ == null) {
+          exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
+        } else {
+          exceptionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000200);
+        result_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000400);
+        stateData_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000800);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_Procedure_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure build() {
+        org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure result = new org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.className_ = className_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.parentId_ = parentId_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.procId_ = procId_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.startTime_ = startTime_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.owner_ = owner_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.state_ = state_;
+        if (((bitField0_ & 0x00000040) == 0x00000040)) {
+          stackId_ = java.util.Collections.unmodifiableList(stackId_);
+          bitField0_ = (bitField0_ & ~0x00000040);
+        }
+        result.stackId_ = stackId_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.lastUpdate_ = lastUpdate_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        result.timeout_ = timeout_;
+        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+          to_bitField0_ |= 0x00000100;
+        }
+        if (exceptionBuilder_ == null) {
+          result.exception_ = exception_;
+        } else {
+          result.exception_ = exceptionBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        result.result_ = result_;
+        if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+          to_bitField0_ |= 0x00000400;
+        }
+        result.stateData_ = stateData_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure.getDefaultInstance()) return this;
+        if (other.hasClassName()) {
+          bitField0_ |= 0x00000001;
+          className_ = other.className_;
+          onChanged();
+        }
+        if (other.hasParentId()) {
+          setParentId(other.getParentId());
+        }
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
+        if (other.hasStartTime()) {
+          setStartTime(other.getStartTime());
+        }
+        if (other.hasOwner()) {
+          bitField0_ |= 0x00000010;
+          owner_ = other.owner_;
+          onChanged();
+        }
+        if (other.hasState()) {
+          setState(other.getState());
+        }
+        if (!other.stackId_.isEmpty()) {
+          if (stackId_.isEmpty()) {
+            stackId_ = other.stackId_;
+            bitField0_ = (bitField0_ & ~0x00000040);
+          } else {
+            ensureStackIdIsMutable();
+            stackId_.addAll(other.stackId_);
+          }
+          onChanged();
+        }
+        if (other.hasLastUpdate()) {
+          setLastUpdate(other.getLastUpdate());
+        }
+        if (other.hasTimeout()) {
+          setTimeout(other.getTimeout());
+        }
+        if (other.hasException()) {
+          mergeException(other.getException());
+        }
+        if (other.hasResult()) {
+          setResult(other.getResult());
+        }
+        if (other.hasStateData()) {
+          setStateData(other.getStateData());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasClassName()) {
+          
+          return false;
+        }
+        if (!hasProcId()) {
+          
+          return false;
+        }
+        if (!hasStartTime()) {
+          
+          return false;
+        }
+        if (!hasState()) {
+          
+          return false;
+        }
+        if (!hasLastUpdate()) {
+          
+          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.ProcedureProtos.Procedure parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.Procedure) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string class_name = 1;
+      private java.lang.Object className_ = "";
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public boolean hasClassName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public java.lang.String getClassName() {
+        java.lang.Object ref = className_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          className_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public com.google.protobuf.ByteString
+          getClassNameBytes() {
+        java.lang.Object ref = className_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          className_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public Builder setClassName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        className_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public Builder clearClassName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        className_ = getDefaultInstance().getClassName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string class_name = 1;</code>
+       *
+       * <pre>
+       * internal "static" state
+       * </pre>
+       */
+      public Builder setClassNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        className_ = value;
+        onChanged();
+        return this;
+      }
+
+      // optional uint64 parent_id = 2;
+      private long parentId_ ;
+      /**
+       * <code>optional uint64 parent_id = 2;</code>
+       *
+       * <pre>
+       * parent if not a root-procedure otherwise not set
+       * </pre>
+       */
+      public boolean hasParentId() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional uint64 parent_id = 2;</code>
+       *
+       * <pre>
+       * parent if not a root-procedure otherwise not set
+       * </pre>
+       */
+      public long getParentId() {
+        return parentId_;
+      }
+      /**
+       * <code>optional uint64 parent_id = 2;</code>
+       *
+       * <pre>
+       * parent if not a root-procedure otherwise not set
+       * </pre>
+       */
+      public Builder setParentId(long value) {
+        bitField0_ |= 0x00000002;
+        parentId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint64 parent_id = 2;</code>
+       *
+       * <pre>
+       * parent if not a root-procedure otherwise not set
+       * </pre>
+       */
+      public Builder clearParentId() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        parentId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 proc_id = 3;
+      private long procId_ ;
+      /**
+       * <code>required uint64 proc_id = 3;</code>
+       */
+      public boolean hasProcId() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required uint64 proc_id = 3;</code>
+       */
+      public long getProcId() {
+        return procId_;
+      }
+      /**
+       * <code>required uint64 proc_id = 3;</code>
+       */
+      public Builder setProcId(long value) {
+        bitField0_ |= 0x00000004;
+        procId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 proc_id = 3;</code>
+       */
+      public Builder clearProcId() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        procId_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // required uint64 start_time = 4;
+      private long startTime_ ;
+      /**
+       * <code>required uint64 start_time = 4;</code>
+       */
+      public boolean hasStartTime() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required uint64 start_time = 4;</code>
+       */
+      public long getStartTime() {
+        return startTime_;
+      }
+      /**
+       * <code>required uint64 start_time = 4;</code>
+       */
+      public Builder setStartTime(long value) {
+        bitField0_ |= 0x00000008;
+        startTime_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 start_time = 4;</code>
+       */
+      public Builder clearStartTime() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        startTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional string owner = 5;
+      private java.lang.Object owner_ = "";
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public boolean hasOwner() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public java.lang.String getOwner() {
+        java.lang.Object ref = owner_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          owner_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public com.google.protobuf.ByteString
+          getOwnerBytes() {
+        java.lang.Object ref = owner_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          owner_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public Builder setOwner(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        owner_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public Builder clearOwner() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        owner_ = getDefaultInstance().getOwner();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string owner = 5;</code>
+       */
+      public Builder setOwnerBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        owner_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required .ProcedureState state = 6;
+      private org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState state_ = org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState.INITIALIZING;
+      /**
+       * <code>required .ProcedureState state = 6;</code>
+       *
+       * <pre>
+       * internal "runtime" state
+       * </pre>
+       */
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>required .ProcedureState state = 6;</code>
+       *
+       * <pre>
+       * internal "runtime" state
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState getState() {
+        return state_;
+      }
+      /**
+       * <code>required .ProcedureState state = 6;</code>
+       *
+       * <pre>
+       * internal "runtime" state
+       * </pre>
+       */
+      public Builder setState(org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000020;
+        state_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required .ProcedureState state = 6;</code>
+       *
+       * <pre>
+       * internal "runtime" state
+       * </pre>
+       */
+      public Builder clearState() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        state_ = org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState.INITIALIZING;
+        onChanged();
+        return this;
+      }
+
+      // repeated uint32 stack_id = 7;
+      private java.util.List<java.lang.Integer> stackId_ = java.util.Collections.emptyList();
+      private void ensureStackIdIsMutable() {
+        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
+          stackId_ = new java.util.ArrayList<java.lang.Integer>(stackId_);
+          bitField0_ |= 0x00000040;
+         }
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public java.util.List<java.lang.Integer>
+          getStackIdList() {
+        return java.util.Collections.unmodifiableList(stackId_);
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public int getStackIdCount() {
+        return stackId_.size();
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public int getStackId(int index) {
+        return stackId_.get(index);
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public Builder setStackId(
+          int index, int value) {
+        ensureStackIdIsMutable();
+        stackId_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public Builder addStackId(int value) {
+        ensureStackIdIsMutable();
+        stackId_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public Builder addAllStackId(
+          java.lang.Iterable<? extends java.lang.Integer> values) {
+        ensureStackIdIsMutable();
+        super.addAll(values, stackId_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated uint32 stack_id = 7;</code>
+       *
+       * <pre>
+       * stack indices in case the procedure was running
+       * </pre>
+       */
+      public Builder clearStackId() {
+        stackId_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000040);
+        onChanged();
+        return this;
+      }
+
+      // required uint64 last_update = 8;
+      private long lastUpdate_ ;
+      /**
+       * <code>required uint64 last_update = 8;</code>
+       */
+      public boolean hasLastUpdate() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>required uint64 last_update = 8;</code>
+       */
+      public long getLastUpdate() {
+        return lastUpdate_;
+      }
+      /**
+       * <code>required uint64 last_update = 8;</code>
+       */
+      public Builder setLastUpdate(long value) {
+        bitField0_ |= 0x00000080;
+        lastUpdate_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 last_update = 8;</code>
+       */
+      public Builder clearLastUpdate() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        lastUpdate_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional uint32 timeout = 9;
+      private int timeout_ ;
+      /**
+       * <code>optional uint32 timeout = 9;</code>
+       */
+      public boolean hasTimeout() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>optional uint32 timeout = 9;</code>
+       */
+      public int getTimeout() {
+        return timeout_;
+      }
+      /**
+       * <code>optional uint32 timeout = 9;</code>
+       */
+      public Builder setTimeout(int value) {
+        bitField0_ |= 0x00000100;
+        timeout_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional uint32 timeout = 9;</code>
+       */
+      public Builder clearTimeout() {
+        bitField0_ = (bitField0_ & ~0x00000100);
+        timeout_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional .ForeignExceptionMessage exception = 10;
+      private org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> exceptionBuilder_;
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public boolean hasException() {
+        return ((bitField0_ & 0x00000200) == 0x00000200);
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage getException() {
+        if (exceptionBuilder_ == null) {
+          return exception_;
+        } else {
+          return exceptionBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public Builder setException(org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage value) {
+        if (exceptionBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          exception_ = value;
+          onChanged();
+        } else {
+          exceptionBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public Builder setException(
+          org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder builderForValue) {
+        if (exceptionBuilder_ == null) {
+          exception_ = builderForValue.build();
+          onChanged();
+        } else {
+          exceptionBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public Builder mergeException(org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage value) {
+        if (exceptionBuilder_ == null) {
+          if (((bitField0_ & 0x00000200) == 0x00000200) &&
+              exception_ != org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance()) {
+            exception_ =
+              org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.newBuilder(exception_).mergeFrom(value).buildPartial();
+          } else {
+            exception_ = value;
+          }
+          onChanged();
+        } else {
+          exceptionBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public Builder clearException() {
+        if (exceptionBuilder_ == null) {
+          exception_ = org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.getDefaultInstance();
+          onChanged();
+        } else {
+          exceptionBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000200);
+        return this;
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder getExceptionBuilder() {
+        bitField0_ |= 0x00000200;
+        onChanged();
+        return getExceptionFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder getExceptionOrBuilder() {
+        if (exceptionBuilder_ != null) {
+          return exceptionBuilder_.getMessageOrBuilder();
+        } else {
+          return exception_;
+        }
+      }
+      /**
+       * <code>optional .ForeignExceptionMessage exception = 10;</code>
+       *
+       * <pre>
+       * user state/results
+       * </pre>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder> 
+          getExceptionFieldBuilder() {
+        if (exceptionBuilder_ == null) {
+          exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage.Builder, org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessageOrBuilder>(
+                  exception_,
+                  getParentForChildren(),
+                  isClean());
+          exception_ = null;
+        }
+        return exceptionBuilder_;
+      }
+
+      // optional bytes result = 11;
+      private com.google.protobuf.ByteString result_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes result = 11;</code>
+       *
+       * <pre>
+       * opaque (user) result structure
+       * </pre>
+       */
+      public boolean hasResult() {
+        return ((bitField0_ & 0x00000400) == 0x00000400);
+      }
+      /**
+       * <code>optional bytes result = 11;</code>
+       *
+       * <pre>
+       * opaque (user) result structure
+       * </pre>
+       */
+      public com.google.protobuf.ByteString getResult() {
+        return result_;
+      }
+      /**
+       * <code>optional bytes result = 11;</code>
+       *
+       * <pre>
+       * opaque (user) result structure
+       * </pre>
+       */
+      public Builder setResult(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000400;
+        result_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes result = 11;</code>
+       *
+       * <pre>
+       * opaque (user) result structure
+       * </pre>
+       */
+      public Builder clearResult() {
+        bitField0_ = (bitField0_ & ~0x00000400);
+        result_ = getDefaultInstance().getResult();
+        onChanged();
+        return this;
+      }
+
+      // optional bytes state_data = 12;
+      private com.google.protobuf.ByteString stateData_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes state_data = 12;</code>
+       *
+       * <pre>
+       * opaque (user) procedure internal-state
+       * </pre>
+       */
+      public boolean hasStateData() {
+        return ((bitField0_ & 0x00000800) == 0x00000800);
+      }
+      /**
+       * <code>optional bytes state_data = 12;</code>
+       *
+       * <pre>
+       * opaque (user) procedure internal-state
+       * </pre>
+       */
+      public com.google.protobuf.ByteString getStateData() {
+        return stateData_;
+      }
+      /**
+       * <code>optional bytes state_data = 12;</code>
+       *
+       * <pre>
+       * opaque (user) procedure internal-state
+       * </pre>
+       */
+      public Builder setStateData(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000800;
+        stateData_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes state_data = 12;</code>
+       *
+       * <pre>
+       * opaque (user) procedure internal-state
+       * </pre>
+       */
+      public Builder clearStateData() {
+        bitField0_ = (bitField0_ & ~0x00000800);
+        stateData_ = getDefaultInstance().getStateData();
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:Procedure)
+    }
+
+    static {
+      defaultInstance = new Procedure(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:Procedure)
+  }
+
+  public interface SequentialProcedureDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required bool executed = 1;
+    /**
+     * <code>required bool executed = 1;</code>
+     */
+    boolean hasExecuted();
+    /**
+     * <code>required bool executed = 1;</code>
+     */
+    boolean getExecuted();
+  }
+  /**
+   * Protobuf type {@code SequentialProcedureData}
+   *
+   * <pre>
+   **
+   * SequentialProcedure data
+   * </pre>
+   */
+  public static final class SequentialProcedureData extends
+      com.google.protobuf.GeneratedMessage
+      implements SequentialProcedureDataOrBuilder {
+    // Use SequentialProcedureData.newBuilder() to construct.
+    private SequentialProcedureData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private SequentialProcedureData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final SequentialProcedureData defaultInstance;
+    public static SequentialProcedureData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public SequentialProcedureData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private SequentialProcedureData(
+        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;
+              executed_ = 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.ProcedureProtos.internal_static_SequentialProcedureData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_SequentialProcedureData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.class, org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<SequentialProcedureData> PARSER =
+        new com.google.protobuf.AbstractParser<SequentialProcedureData>() {
+      public SequentialProcedureData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new SequentialProcedureData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<SequentialProcedureData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required bool executed = 1;
+    public static final int EXECUTED_FIELD_NUMBER = 1;
+    private boolean executed_;
+    /**
+     * <code>required bool executed = 1;</code>
+     */
+    public boolean hasExecuted() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bool executed = 1;</code>
+     */
+    public boolean getExecuted() {
+      return executed_;
+    }
+
+    private void initFields() {
+      executed_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasExecuted()) {
+        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, executed_);
+      }
+      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, executed_);
+      }
+      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.ProcedureProtos.SequentialProcedureData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData other = (org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData) obj;
+
+      boolean result = true;
+      result = result && (hasExecuted() == other.hasExecuted());
+      if (hasExecuted()) {
+        result = result && (getExecuted()
+            == other.getExecuted());
+      }
+      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 (hasExecuted()) {
+        hash = (37 * hash) + EXECUTED_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getExecuted());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData 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.ProcedureProtos.SequentialProcedureData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData 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.ProcedureProtos.SequentialProcedureData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData 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.ProcedureProtos.SequentialProcedureData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData 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.ProcedureProtos.SequentialProcedureData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData 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.ProcedureProtos.SequentialProcedureData 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 SequentialProcedureData}
+     *
+     * <pre>
+     **
+     * SequentialProcedure data
+     * </pre>
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_SequentialProcedureData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.internal_static_SequentialProcedureData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.class, org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.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();
+        executed_ = false;
+        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.ProcedureProtos.internal_static_SequentialProcedureData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData build() {
+        org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData result = new org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.executed_ = executed_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData.getDefaultInstance()) return this;
+        if (other.hasExecuted()) {
+          setExecuted(other.getExecuted());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasExecuted()) {
+          
+          return false;
+        }
+        return tr

<TRUNCATED>

[27/28] 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/67a785d9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/67a785d9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/67a785d9

Branch: refs/heads/hbase-12439
Commit: 67a785d979d833555c829eefca4e1d022a93bc74
Parents: c5bf41a
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Thu Apr 9 21:52:02 2015 +0100
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Fri Apr 10 08:55:06 2015 +0100

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



[16/28] 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/b9c4b8f6/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..98260c1 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();
@@ -1402,7 +1529,7 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:CreateTableStateData)
   }
 
-  public interface DeleteTableStateDataOrBuilder
+  public interface ModifyTableStateDataOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
     // required .UserInformation user_info = 1;
@@ -1419,64 +1546,63 @@ public final class MasterProcedureProtos {
      */
     org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
 
-    // required .TableName table_name = 2;
+    // optional .TableSchema unmodified_table_schema = 2;
     /**
-     * <code>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    boolean hasTableName();
+    boolean hasUnmodifiedTableSchema();
     /**
-     * <code>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema();
     /**
-     * <code>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder();
 
-    // repeated .RegionInfo region_info = 3;
+    // required .TableSchema modified_table_schema = 3;
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> 
-        getRegionInfoList();
+    boolean hasModifiedTableSchema();
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index);
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema();
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    int getRegionInfoCount();
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder();
+
+    // required bool delete_column_family_in_modify = 4;
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required bool delete_column_family_in_modify = 4;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
-        getRegionInfoOrBuilderList();
+    boolean hasDeleteColumnFamilyInModify();
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required bool delete_column_family_in_modify = 4;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
-        int index);
+    boolean getDeleteColumnFamilyInModify();
   }
   /**
-   * Protobuf type {@code DeleteTableStateData}
+   * Protobuf type {@code ModifyTableStateData}
    */
-  public static final class DeleteTableStateData extends
+  public static final class ModifyTableStateData extends
       com.google.protobuf.GeneratedMessage
-      implements DeleteTableStateDataOrBuilder {
-    // Use DeleteTableStateData.newBuilder() to construct.
-    private DeleteTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      implements ModifyTableStateDataOrBuilder {
+    // Use ModifyTableStateData.newBuilder() to construct.
+    private ModifyTableStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
       super(builder);
       this.unknownFields = builder.getUnknownFields();
     }
-    private DeleteTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+    private ModifyTableStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
 
-    private static final DeleteTableStateData defaultInstance;
-    public static DeleteTableStateData getDefaultInstance() {
+    private static final ModifyTableStateData defaultInstance;
+    public static ModifyTableStateData getDefaultInstance() {
       return defaultInstance;
     }
 
-    public DeleteTableStateData getDefaultInstanceForType() {
+    public ModifyTableStateData getDefaultInstanceForType() {
       return defaultInstance;
     }
 
@@ -1486,7 +1612,7 @@ public final class MasterProcedureProtos {
         getUnknownFields() {
       return this.unknownFields;
     }
-    private DeleteTableStateData(
+    private ModifyTableStateData(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
@@ -1523,24 +1649,34 @@ public final class MasterProcedureProtos {
               break;
             }
             case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
               if (((bitField0_ & 0x00000002) == 0x00000002)) {
-                subBuilder = tableName_.toBuilder();
+                subBuilder = unmodifiedTableSchema_.toBuilder();
               }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              unmodifiedTableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
               if (subBuilder != null) {
-                subBuilder.mergeFrom(tableName_);
-                tableName_ = subBuilder.buildPartial();
+                subBuilder.mergeFrom(unmodifiedTableSchema_);
+                unmodifiedTableSchema_ = 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;
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = modifiedTableSchema_.toBuilder();
               }
-              regionInfo_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.PARSER, extensionRegistry));
+              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;
             }
           }
@@ -1551,37 +1687,34 @@ public final class MasterProcedureProtos {
         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;
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_descriptor;
     }
 
     protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
         internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_fieldAccessorTable
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_fieldAccessorTable
           .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.Builder.class);
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.Builder.class);
     }
 
-    public static com.google.protobuf.Parser<DeleteTableStateData> PARSER =
-        new com.google.protobuf.AbstractParser<DeleteTableStateData>() {
-      public DeleteTableStateData parsePartialFrom(
+    public static com.google.protobuf.Parser<ModifyTableStateData> PARSER =
+        new com.google.protobuf.AbstractParser<ModifyTableStateData>() {
+      public ModifyTableStateData parsePartialFrom(
           com.google.protobuf.CodedInputStream input,
           com.google.protobuf.ExtensionRegistryLite extensionRegistry)
           throws com.google.protobuf.InvalidProtocolBufferException {
-        return new DeleteTableStateData(input, extensionRegistry);
+        return new ModifyTableStateData(input, extensionRegistry);
       }
     };
 
     @java.lang.Override
-    public com.google.protobuf.Parser<DeleteTableStateData> getParserForType() {
+    public com.google.protobuf.Parser<ModifyTableStateData> getParserForType() {
       return PARSER;
     }
 
@@ -1608,68 +1741,71 @@ public final class MasterProcedureProtos {
       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_;
+    // 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>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    public boolean hasTableName() {
+    public boolean hasUnmodifiedTableSchema() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
-      return tableName_;
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+      return unmodifiedTableSchema_;
     }
     /**
-     * <code>required .TableName table_name = 2;</code>
+     * <code>optional .TableSchema unmodified_table_schema = 2;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
-      return tableName_;
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+      return unmodifiedTableSchema_;
     }
 
-    // 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_;
+    // 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>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    public java.util.List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo> getRegionInfoList() {
-      return regionInfo_;
+    public boolean hasModifiedTableSchema() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder> 
-        getRegionInfoOrBuilderList() {
-      return regionInfo_;
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getModifiedTableSchema() {
+      return modifiedTableSchema_;
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required .TableSchema modified_table_schema = 3;</code>
      */
-    public int getRegionInfoCount() {
-      return regionInfo_.size();
+    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>repeated .RegionInfo region_info = 3;</code>
+     * <code>required bool delete_column_family_in_modify = 4;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo getRegionInfo(int index) {
-      return regionInfo_.get(index);
+    public boolean hasDeleteColumnFamilyInModify() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>repeated .RegionInfo region_info = 3;</code>
+     * <code>required bool delete_column_family_in_modify = 4;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfoOrBuilder getRegionInfoOrBuilder(
-        int index) {
-      return regionInfo_.get(index);
+    public boolean getDeleteColumnFamilyInModify() {
+      return deleteColumnFamilyInModify_;
     }
 
     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();
+      unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      deleteColumnFamilyInModify_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -1680,24 +1816,28 @@ public final class MasterProcedureProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!hasTableName()) {
+      if (!hasModifiedTableSchema()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!getUserInfo().isInitialized()) {
+      if (!hasDeleteColumnFamilyInModify()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (!getTableName().isInitialized()) {
+      if (!getUserInfo().isInitialized()) {
         memoizedIsInitialized = 0;
         return false;
       }
-      for (int i = 0; i < getRegionInfoCount(); i++) {
-        if (!getRegionInfo(i).isInitialized()) {
+      if (hasUnmodifiedTableSchema()) {
+        if (!getUnmodifiedTableSchema().isInitialized()) {
           memoizedIsInitialized = 0;
           return false;
         }
       }
+      if (!getModifiedTableSchema().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -1709,10 +1849,13 @@ public final class MasterProcedureProtos {
         output.writeMessage(1, userInfo_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeMessage(2, tableName_);
+        output.writeMessage(2, unmodifiedTableSchema_);
       }
-      for (int i = 0; i < regionInfo_.size(); i++) {
-        output.writeMessage(3, regionInfo_.get(i));
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(3, modifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(4, deleteColumnFamilyInModify_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -1729,11 +1872,15 @@ public final class MasterProcedureProtos {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, tableName_);
+          .computeMessageSize(2, unmodifiedTableSchema_);
       }
-      for (int i = 0; i < regionInfo_.size(); i++) {
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, regionInfo_.get(i));
+          .computeMessageSize(3, modifiedTableSchema_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(4, deleteColumnFamilyInModify_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -1752,10 +1899,10 @@ public final class MasterProcedureProtos {
       if (obj == this) {
        return true;
       }
-      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData)) {
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData)) {
         return super.equals(obj);
       }
-      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData) obj;
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) obj;
 
       boolean result = true;
       result = result && (hasUserInfo() == other.hasUserInfo());
@@ -1763,13 +1910,21 @@ public final class MasterProcedureProtos {
         result = result && getUserInfo()
             .equals(other.getUserInfo());
       }
-      result = result && (hasTableName() == other.hasTableName());
-      if (hasTableName()) {
-        result = result && getTableName()
-            .equals(other.getTableName());
+      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());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1787,66 +1942,70 @@ public final class MasterProcedureProtos {
         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 (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();
+      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());
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
     }
 
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.ByteString data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.ByteString data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(byte[] data)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(byte[] data)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         byte[] data,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws com.google.protobuf.InvalidProtocolBufferException {
       return PARSER.parseFrom(data, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseDelimitedFrom(java.io.InputStream input)
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(java.io.InputStream input)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseDelimitedFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseDelimitedFrom(
         java.io.InputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
       return PARSER.parseDelimitedFrom(input, extensionRegistry);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.CodedInputStream input)
         throws java.io.IOException {
       return PARSER.parseFrom(input);
     }
-    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData parseFrom(
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData parseFrom(
         com.google.protobuf.CodedInputStream input,
         com.google.protobuf.ExtensionRegistryLite extensionRegistry)
         throws java.io.IOException {
@@ -1855,7 +2014,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.DeleteTableStateData prototype) {
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData prototype) {
       return newBuilder().mergeFrom(prototype);
     }
     public Builder toBuilder() { return newBuilder(this); }
@@ -1867,24 +2026,24 @@ public final class MasterProcedureProtos {
       return builder;
     }
     /**
-     * Protobuf type {@code DeleteTableStateData}
+     * Protobuf type {@code ModifyTableStateData}
      */
     public static final class Builder extends
         com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateDataOrBuilder {
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateDataOrBuilder {
       public static final com.google.protobuf.Descriptors.Descriptor
           getDescriptor() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_descriptor;
       }
 
       protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
           internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_fieldAccessorTable
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_fieldAccessorTable
             .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.Builder.class);
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.Builder.class);
       }
 
-      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData.newBuilder()
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.newBuilder()
       private Builder() {
         maybeForceBuilderInitialization();
       }
@@ -1897,8 +2056,8 @@ public final class MasterProcedureProtos {
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getUserInfoFieldBuilder();
-          getTableNameFieldBuilder();
-          getRegionInfoFieldBuilder();
+          getUnmodifiedTableSchemaFieldBuilder();
+          getModifiedTableSchemaFieldBuilder();
         }
       }
       private static Builder create() {
@@ -1913,18 +2072,20 @@ public final class MasterProcedureProtos {
           userInfoBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        if (tableNameBuilder_ == null) {
-          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
         } else {
-          tableNameBuilder_.clear();
+          unmodifiedTableSchemaBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000002);
-        if (regionInfoBuilder_ == null) {
-          regionInfo_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000004);
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
         } else {
-          regionInfoBuilder_.clear();
+          modifiedTableSchemaBuilder_.clear();
         }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        deleteColumnFamilyInModify_ = false;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -1934,23 +2095,23 @@ public final class MasterProcedureProtos {
 
       public com.google.protobuf.Descriptors.Descriptor
           getDescriptorForType() {
-        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_DeleteTableStateData_descriptor;
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyTableStateData_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.ModifyTableStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.getDefaultInstance();
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData build() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData result = buildPartial();
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData result = buildPartial();
         if (!result.isInitialized()) {
           throw newUninitializedMessageException(result);
         }
         return result;
       }
 
-      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData buildPartial() {
-        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableStateData(this);
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData(this);
         int from_bitField0_ = bitField0_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@@ -1964,28 +2125,1083 @@ public final class MasterProcedureProtos {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        if (tableNameBuilder_ == null) {
-          result.tableName_ = tableName_;
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchema_;
         } else {
-          result.tableName_ = tableNameBuilder_.build();
+          result.unmodifiedTableSchema_ = unmodifiedTableSchemaBuilder_.build();
         }
-        if (regionInfoBuilder_ == null) {
-          if (((bitField0_ & 0x00000004) == 0x00000004)) {
-            regionInfo_ = java.util.Collections.unmodifiableList(regionInfo_);
-            bitField0_ = (bitField0_ & ~0x00000004);
-          }
-          result.regionInfo_ = regionInfo_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (modifiedTableSchemaBuilder_ == null) {
+          result.modifiedTableSchema_ = modifiedTableSchema_;
         } else {
-          result.regionInfo_ = regionInfoBuilder_.build();
+          result.modifiedTableSchema_ = modifiedTableSchemaBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
         }
+        result.deleteColumnFamilyInModify_ = deleteColumnFamilyInModify_;
         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);
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasUnmodifiedTableSchema()) {
+          mergeUnmodifiedTableSchema(other.getUnmodifiedTableSchema());
+        }
+        if (other.hasModifiedTableSchema()) {
+          mergeModifiedTableSchema(other.getModifiedTableSchema());
+        }
+        if (other.hasDeleteColumnFamilyInModify()) {
+          setDeleteColumnFamilyInModify(other.getDeleteColumnFamilyInModify());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasUserInfo()) {
+          
+          return false;
+        }
+        if (!hasModifiedTableSchema()) {
+          
+          return false;
+        }
+        if (!hasDeleteColumnFamilyInModify()) {
+          
+          return false;
+        }
+        if (!getUserInfo().isInitialized()) {
+          
+          return false;
+        }
+        if (hasUnmodifiedTableSchema()) {
+          if (!getUnmodifiedTableSchema().isInitialized()) {
+            
+            return false;
+          }
+        }
+        if (!getModifiedTableSchema().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.ModifyTableStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyTableStateData) 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_;
+      }
+
+      // 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> unmodifiedTableSchemaBuilder_;
+      /**
+       * <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() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          return unmodifiedTableSchema_;
+        } else {
+          return unmodifiedTableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</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_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</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_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      public Builder mergeUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              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_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      public Builder clearUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 2;</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 = 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> 
+          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_;
+      }
+
+      // required .TableSchema modified_table_schema = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema modifiedTableSchema_ = 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> modifiedTableSchemaBuilder_;
+      /**
+       * <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() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          return modifiedTableSchema_;
+        } else {
+          return modifiedTableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder setModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          modifiedTableSchema_ = value;
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder setModifiedTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder mergeModifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (modifiedTableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              modifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            modifiedTableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(modifiedTableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            modifiedTableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public Builder clearModifiedTableSchema() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          modifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getModifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getModifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getModifiedTableSchemaOrBuilder() {
+        if (modifiedTableSchemaBuilder_ != null) {
+          return modifiedTableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return modifiedTableSchema_;
+        }
+      }
+      /**
+       * <code>required .TableSchema modified_table_schema = 3;</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> 
+          getModifiedTableSchemaFieldBuilder() {
+        if (modifiedTableSchemaBuilder_ == null) {
+          modifiedTableSchemaBuilder_ = 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>(
+                  modifiedTableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          modifiedTableSchema_ = null;
+        }
+        return modifiedTableSchemaBuilder_;
+      }
+
+      // required bool delete_column_family_in_modify = 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_;
+      }
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public Builder setDeleteColumnFamilyInModify(boolean value) {
+        bitField0_ |= 0x00000008;
+        deleteColumnFamilyInModify_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bool delete_column_family_in_modify = 4;</code>
+       */
+      public Builder clearDeleteColumnFamilyInModify() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        deleteColumnFamilyInModify_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ModifyTableStateData)
+    }
+
+    static {
+      defaultInstance = new ModifyTableStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ModifyTableStateData)
+  }
+
+  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;
@@ -2566,6 +3782,11 @@ public final class MasterProcedureProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_CreateTableStateData_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ModifyTableStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ModifyTableStateData_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_DeleteTableStateData_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -2583,23 +3804,35 @@ public final class MasterProcedureProtos {
       "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\"}\n\024DeleteTableStateD" +
-      "ata\022#\n\tuser_info\030\001 \002(\0132\020.UserInformation" +
-      "\022\036\n\ntable_name\030\002 \002(\0132\n.TableName\022 \n\013regi" +
-      "on_info\030\003 \003(\0132\013.RegionInfo*\330\001\n\020CreateTab" +
-      "leState\022\036\n\032CREATE_TABLE_PRE_OPERATION\020\001\022" +
-      " \n\034CREATE_TABLE_WRITE_FS_LAYOUT\020\002\022\034\n\030CRE",
-      "ATE_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_OPERAT" +
-      "ION\020\006*\337\001\n\020DeleteTableState\022\036\n\032DELETE_TAB" +
-      "LE_PRE_OPERATION\020\001\022!\n\035DELETE_TABLE_REMOV" +
-      "E_FROM_META\020\002\022 \n\034DELETE_TABLE_CLEAR_FS_L" +
-      "AYOUT\020\003\022\"\n\036DELETE_TABLE_UPDATE_DESC_CACH" +
-      "E\020\004\022!\n\035DELETE_TABLE_UNASSIGN_REGIONS\020\005\022\037" +
-      "\n\033DELETE_TABLE_POST_OPERATION\020\006BK\n*org.a" +
-      "pache.hadoop.hbase.protobuf.generatedB\025M",
-      "asterProcedureProtosH\001\210\001\001\240\001\001"
+      "\030\003 \003(\0132\013.RegionInfo\"\277\001\n\024ModifyTableState" +
+      "Data\022#\n\tuser_info\030\001 \002(\0132\020.UserInformatio" +
+      "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
+      "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
+      "\014.TableSchema\022&\n\036delete_column_family_in" +
+      "_modify\030\004 \002(\010\"}\n\024DeleteTableStateData\022#\n",
+      "\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\nta" +
+      "ble_name\030\002 \002(\0132\n.TableName\022 \n\013region_inf" +
+      "o\030\003 \003(\0132\013.RegionInfo*\330\001\n\020CreateTableStat" +
+      "e\022\036\n\032CREATE_TABLE_PRE_OPERATION\020\001\022 \n\034CRE" +
+      "ATE_TABLE_WRITE_FS_LAYOUT\020\002\022\034\n\030CREATE_TA" +
+      "BLE_ADD_TO_META\020\003\022\037\n\033CREATE_TABLE_ASSIGN" +
+      "_REGIONS\020\004\022\"\n\036CREATE_TABLE_UPDATE_DESC_C" +
+      "ACHE\020\005\022\037\n\033CREATE_TABLE_POST_OPERATION\020\006*" +
+      "\207\002\n\020ModifyTableState\022\030\n\024MODIFY_TABLE_PRE" +
+      "PARE\020\001\022\036\n\032MODIFY_TABLE_PRE_OPERATION\020\002\022(",
+      "\n$MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR\020\003" +
+      "\022&\n\"MODIFY_TABLE_REMOVE_REPLICA_COLUMN\020\004" +
+      "\022!\n\035MODIFY_TABLE_DELETE_FS_LAYOUT\020\005\022\037\n\033M" +
+      "ODIFY_TABLE_POST_OPERATION\020\006\022#\n\037MODIFY_T" +
+      "ABLE_REOPEN_ALL_REGIONS\020\007*\337\001\n\020DeleteTabl" +
+      "eState\022\036\n\032DELETE_TABLE_PRE_OPERATION\020\001\022!" +
+      "\n\035DELETE_TABLE_REMOVE_FROM_META\020\002\022 \n\034DEL" +
+      "ETE_TABLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036DELETE_TA" +
+      "BLE_UPDATE_DESC_CACHE\020\004\022!\n\035DELETE_TABLE_" +
+      "UNASSIGN_REGIONS\020\005\022\037\n\033DELETE_TABLE_POST_",
+      "OPERATION\020\006BK\n*org.apache.hadoop.hbase.p" +
+      "rotobuf.generatedB\025MasterProcedureProtos" +
+      "H\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -2612,8 +3845,14 @@ public final class MasterProcedureProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CreateTableStateData_descriptor,
               new java.lang.String[] { "UserInfo", "TableSchema", "RegionInfo", });
-          internal_static_DeleteTableStateData_descriptor =
+          internal_static_ModifyTableStateData_descriptor =
             getDescriptor().getMessageTypes().get(1);
+          internal_static_ModifyTableStateData_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ModifyTableStateData_descriptor,
+              new ja

<TRUNCATED>

[15/28] 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/b9c4b8f6/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..97d1af6 100644
--- a/hbase-protocol/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol/src/main/protobuf/MasterProcedure.proto
@@ -58,6 +58,23 @@ message CreateTableStateData {
   repeated RegionInfo region_info = 3;
 }
 
+enum ModifyTableState {
+  MODIFY_TABLE_PREPARE = 1;
+  MODIFY_TABLE_PRE_OPERATION = 2;
+  MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR = 3;
+  MODIFY_TABLE_REMOVE_REPLICA_COLUMN = 4;
+  MODIFY_TABLE_DELETE_FS_LAYOUT = 5;
+  MODIFY_TABLE_POST_OPERATION = 6;
+  MODIFY_TABLE_REOPEN_ALL_REGIONS = 7;
+}
+
+message ModifyTableStateData {
+  required UserInformation user_info = 1;
+  optional TableSchema unmodified_table_schema = 2;
+  required TableSchema modified_table_schema = 3;
+  required bool delete_column_family_in_modify = 4;
+}
+
 enum DeleteTableState {
   DELETE_TABLE_PRE_OPERATION = 1;
   DELETE_TABLE_REMOVE_FROM_META = 2;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c4b8f6/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/b9c4b8f6/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/b9c4b8f6/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/b9c4b8f6/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..8ccdaa4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -0,0 +1,512 @@
+/**
+ * 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 final AtomicBoolean aborted = new AtomicBoolean(false);
+
+  private HTableDescriptor unmodifiedHTableDescriptor = null;
+  private HTableDescriptor modifiedHTableDescriptor;
+  private UserGroupInformation user;
+  private boolean deleteColumnFamilyInModify;
+
+  private List<HRegionInfo> regionInfoList;
+  private Boolean traceEnabled = null;
+
+  public ModifyTableProcedure() {
+    initilize();
+  }
+
+  public ModifyTableProcedure(
+    final MasterProcedureEnv env,
+    final HTableDescriptor htd) throws IOException {
+    initilize();
+    this.modifiedHTableDescriptor = htd;
+    this.user = env.getRequestUser().getUGI();
+  }
+
+  private void initilize() {
+    this.unmodifiedHTableDescriptor = null;
+    this.regionInfoList = null;
+    this.traceEnabled = null;
+    this.deleteColumnFamilyInModify = false;
+  }
+
+  @Override
+  protected Flow executeFromState(final MasterProcedureEnv env, final ModifyTableState state) {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " execute state=" + state);
+    }
+
+    try {
+      switch (state) {
+      case MODIFY_TABLE_PREPARE:
+        prepareModify(env);
+        setNextState(ModifyTableState.MODIFY_TABLE_PRE_OPERATION);
+        break;
+      case MODIFY_TABLE_PRE_OPERATION:
+        preModify(env, state);
+        setNextState(ModifyTableState.MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR);
+        break;
+      case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR:
+        updateTableDescriptor(env);
+        setNextState(ModifyTableState.MODIFY_TABLE_REMOVE_REPLICA_COLUMN);
+        break;
+      case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
+        updateReplicaColumnsIfNeeded(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
+        if (deleteColumnFamilyInModify) {
+          setNextState(ModifyTableState.MODIFY_TABLE_DELETE_FS_LAYOUT);
+        } else {
+          setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
+        }
+        break;
+      case MODIFY_TABLE_DELETE_FS_LAYOUT:
+        deleteFromFs(env, unmodifiedHTableDescriptor, modifiedHTableDescriptor);
+        setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
+        break;
+      case MODIFY_TABLE_POST_OPERATION:
+        postModify(env, state);
+        setNextState(ModifyTableState.MODIFY_TABLE_REOPEN_ALL_REGIONS);
+        break;
+      case MODIFY_TABLE_REOPEN_ALL_REGIONS:
+        reOpenAllRegionsIfTableIsOnline(env);
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (InterruptedException|IOException e) {
+      if (!isRollbackSupported(state)) {
+        // We reach a state that cannot be rolled back. We just need to keep retry.
+        LOG.warn("Error trying to modify table=" + getTableName() + " state=" + state, e);
+      } else {
+        LOG.error("Error trying to modify table=" + getTableName() + " state=" + state, e);
+        setFailure("master-modify-table", e);
+      }
+    }
+    return Flow.HAS_MORE_STATE;
+  }
+
+  @Override
+  protected void rollbackState(final MasterProcedureEnv env, final ModifyTableState state)
+      throws IOException {
+    if (isTraceEnabled()) {
+      LOG.trace(this + " rollback state=" + state);
+    }
+    try {
+      switch (state) {
+      case MODIFY_TABLE_REOPEN_ALL_REGIONS:
+        break; // Nothing to undo.
+      case MODIFY_TABLE_POST_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to un-modify?
+        break;
+      case MODIFY_TABLE_DELETE_FS_LAYOUT:
+        // Once we reach to this state - we could NOT rollback - as it is tricky to undelete
+        // the deleted files. We are not suppose to reach here, throw exception so that we know
+        // there is a code bug to investigate.
+        assert deleteColumnFamilyInModify;
+        throw new UnsupportedOperationException(this + " rollback of state=" + state
+            + " is unsupported.");
+      case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
+        // Undo the replica column update.
+        updateReplicaColumnsIfNeeded(env, modifiedHTableDescriptor, unmodifiedHTableDescriptor);
+        break;
+      case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR:
+        restoreTableDescriptor(env);
+        break;
+      case MODIFY_TABLE_PRE_OPERATION:
+        // TODO-MAYBE: call the coprocessor event to un-modify?
+        break;
+      case MODIFY_TABLE_PREPARE:
+        break; // Nothing to undo.
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+      }
+    } catch (IOException e) {
+      LOG.warn("Fail trying to rollback modify table=" + getTableName() + " state=" + state, e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected ModifyTableState getState(final int stateId) {
+    return ModifyTableState.valueOf(stateId);
+  }
+
+  @Override
+  protected int getStateId(final ModifyTableState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected ModifyTableState getInitialState() {
+    return ModifyTableState.MODIFY_TABLE_PREPARE;
+  }
+
+  @Override
+  protected void setNextState(final ModifyTableState state) {
+    if (aborted.get() && isRollbackSupported(state)) {
+      setAbortFailure("modify-table", "abort requested");
+    } else {
+      super.setNextState(state);
+    }
+  }
+
+  @Override
+  public boolean abort(final MasterProcedureEnv env) {
+    aborted.set(true);
+    return true;
+  }
+
+  @Override
+  protected boolean acquireLock(final MasterProcedureEnv env) {
+    if (!env.isInitialized()) return false;
+    return env.getProcedureQueue().tryAcquireTableWrite(
+      getTableName(),
+      EventType.C_M_MODIFY_TABLE.toString());
+  }
+
+  @Override
+  protected void releaseLock(final MasterProcedureEnv env) {
+    env.getProcedureQueue().releaseTableWrite(getTableName());
+  }
+
+  @Override
+  public void serializeStateData(final OutputStream stream) throws IOException {
+    super.serializeStateData(stream);
+
+    MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
+        MasterProcedureProtos.ModifyTableStateData.newBuilder()
+            .setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
+            .setModifiedTableSchema(modifiedHTableDescriptor.convert())
+            .setDeleteColumnFamilyInModify(deleteColumnFamilyInModify);
+
+    if (unmodifiedHTableDescriptor != null) {
+      modifyTableMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
+    }
+
+    modifyTableMsg.build().writeDelimitedTo(stream);
+  }
+
+  @Override
+  public void deserializeStateData(final InputStream stream) throws IOException {
+    super.deserializeStateData(stream);
+
+    MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
+        MasterProcedureProtos.ModifyTableStateData.parseDelimitedFrom(stream);
+    user = MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo());
+    modifiedHTableDescriptor = HTableDescriptor.convert(modifyTableMsg.getModifiedTableSchema());
+    deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
+
+    if (modifyTableMsg.hasUnmodifiedTableSchema()) {
+      unmodifiedHTableDescriptor =
+          HTableDescriptor.convert(modifyTableMsg.getUnmodifiedTableSchema());
+    }
+  }
+
+  @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");
+      }
+    }
+
+    // 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(),
+          getRegionInfoList(env),
+          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(), getRegionInfoList(env))) {
+      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;
+  }
+
+  private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
+    if (regionInfoList == null) {
+      regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
+    }
+    return regionInfoList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b9c4b8f6/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();
+  }
+}


[18/28] 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/7af69304/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/7af69304/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/7af69304/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/7af69304/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/7af69304/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
index ad5e671..84e9bef 100644
--- 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
@@ -125,6 +125,7 @@ public class DeleteTableProcedure
           LOG.debug("delete '" + getTableName() + "' from filesystem");
           DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
           setNextState(DeleteTableState.DELETE_TABLE_UPDATE_DESC_CACHE);
+          regions = null;
           break;
         case DELETE_TABLE_UPDATE_DESC_CACHE:
           LOG.debug("delete '" + getTableName() + "' descriptor");

http://git-wip-us.apache.org/repos/asf/hbase/blob/7af69304/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 { }


[21/28] 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/c5bf41a7/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/c5bf41a7/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();
+  }
+}


[23/28] 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/c5bf41a7/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 98260c1..4713a0a 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 {
 
@@ -3776,66 +4112,3621 @@ public final class MasterProcedureProtos {
     // @@protoc_insertion_point(class_scope:DeleteTableStateData)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_CreateTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_CreateTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_ModifyTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_ModifyTableStateData_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_DeleteTableStateData_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_DeleteTableStateData_fieldAccessorTable;
+  public interface AddColumnFamilyStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\025MasterProcedure.proto\032\013HBase.proto\032\tRP" +
-      "C.proto\"\201\001\n\024CreateTableStateData\022#\n\tuser" +
-      "_info\030\001 \002(\0132\020.UserInformation\022\"\n\014table_s" +
-      "chema\030\002 \002(\0132\014.TableSchema\022 \n\013region_info" +
-      "\030\003 \003(\0132\013.RegionInfo\"\277\001\n\024ModifyTableState" +
-      "Data\022#\n\tuser_info\030\001 \002(\0132\020.UserInformatio" +
-      "n\022-\n\027unmodified_table_schema\030\002 \001(\0132\014.Tab" +
-      "leSchema\022+\n\025modified_table_schema\030\003 \002(\0132" +
-      "\014.TableSchema\022&\n\036delete_column_family_in" +
-      "_modify\030\004 \002(\010\"}\n\024DeleteTableStateData\022#\n",
-      "\tuser_info\030\001 \002(\0132\020.UserInformation\022\036\n\nta" +
-      "ble_name\030\002 \002(\0132\n.TableName\022 \n\013region_inf" +
-      "o\030\003 \003(\0132\013.RegionInfo*\330\001\n\020CreateTableStat" +
-      "e\022\036\n\032CREATE_TABLE_PRE_OPERATION\020\001\022 \n\034CRE" +
-      "ATE_TABLE_WRITE_FS_LAYOUT\020\002\022\034\n\030CREATE_TA" +
-      "BLE_ADD_TO_META\020\003\022\037\n\033CREATE_TABLE_ASSIGN" +
-      "_REGIONS\020\004\022\"\n\036CREATE_TABLE_UPDATE_DESC_C" +
-      "ACHE\020\005\022\037\n\033CREATE_TABLE_POST_OPERATION\020\006*" +
-      "\207\002\n\020ModifyTableState\022\030\n\024MODIFY_TABLE_PRE" +
-      "PARE\020\001\022\036\n\032MODIFY_TABLE_PRE_OPERATION\020\002\022(",
-      "\n$MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR\020\003" +
-      "\022&\n\"MODIFY_TABLE_REMOVE_REPLICA_COLUMN\020\004" +
-      "\022!\n\035MODIFY_TABLE_DELETE_FS_LAYOUT\020\005\022\037\n\033M" +
-      "ODIFY_TABLE_POST_OPERATION\020\006\022#\n\037MODIFY_T" +
-      "ABLE_REOPEN_ALL_REGIONS\020\007*\337\001\n\020DeleteTabl" +
-      "eState\022\036\n\032DELETE_TABLE_PRE_OPERATION\020\001\022!" +
-      "\n\035DELETE_TABLE_REMOVE_FROM_META\020\002\022 \n\034DEL" +
-      "ETE_TABLE_CLEAR_FS_LAYOUT\020\003\022\"\n\036DELETE_TA" +
-      "BLE_UPDATE_DESC_CACHE\020\004\022!\n\035DELETE_TABLE_" +
-      "UNASSIGN_REGIONS\020\005\022\037\n\033DELETE_TABLE_POST_",
-      "OPERATION\020\006BK\n*org.apache.hadoop.hbase.p" +
-      "rotobuf.generatedB\025MasterProcedureProtos" +
-      "H\001\210\001\001\240\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+    // 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();
+  }
+  /**
+   * Protobuf type {@code AddColumnFamilyStateData}
+   */
+  public static final class AddColumnFamilyStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements AddColumnFamilyStateDataOrBuilder {
+    // Use AddColumnFamilyStateData.newBuilder() to construct.
+    private AddColumnFamilyStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private AddColumnFamilyStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final AddColumnFamilyStateData defaultInstance;
+    public static AddColumnFamilyStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public AddColumnFamilyStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private AddColumnFamilyStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 26: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = columnfamilySchema_.toBuilder();
+              }
+              columnfamilySchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(columnfamilySchema_);
+                columnfamilySchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
+            case 34: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000008) == 0x00000008)) {
+                subBuilder = unmodifiedTableSchema_.toBuilder();
+              }
+              unmodifiedTableSchema_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(unmodifiedTableSchema_);
+                unmodifiedTableSchema_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000008;
+              break;
+            }
+          }
+        }
+      } 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_AddColumnFamilyStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<AddColumnFamilyStateData> PARSER =
+        new com.google.protobuf.AbstractParser<AddColumnFamilyStateData>() {
+      public AddColumnFamilyStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new AddColumnFamilyStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<AddColumnFamilyStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .UserInformation user_info = 1;
+    public static final int USER_INFO_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public boolean hasUserInfo() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+      return userInfo_;
+    }
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+      return userInfo_;
+    }
+
+    // required .TableName table_name = 2;
+    public static final int TABLE_NAME_FIELD_NUMBER = 2;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // required .ColumnFamilySchema columnfamily_schema = 3;
+    public static final int COLUMNFAMILY_SCHEMA_FIELD_NUMBER = 3;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_;
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public boolean hasColumnfamilySchema() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema() {
+      return columnfamilySchema_;
+    }
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder() {
+      return columnfamilySchema_;
+    }
+
+    // optional .TableSchema unmodified_table_schema = 4;
+    public static final int UNMODIFIED_TABLE_SCHEMA_FIELD_NUMBER = 4;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public boolean hasUnmodifiedTableSchema() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+      return unmodifiedTableSchema_;
+    }
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+      return unmodifiedTableSchema_;
+    }
+
+    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();
+    }
+    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;
+        }
+      }
+      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_);
+      }
+      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_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) obj;
+
+      boolean result = true;
+      result = result && (hasUserInfo() == other.hasUserInfo());
+      if (hasUserInfo()) {
+        result = result && getUserInfo()
+            .equals(other.getUserInfo());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasColumnfamilySchema() == other.hasColumnfamilySchema());
+      if (hasColumnfamilySchema()) {
+        result = result && getColumnfamilySchema()
+            .equals(other.getColumnfamilySchema());
+      }
+      result = result && (hasUnmodifiedTableSchema() == other.hasUnmodifiedTableSchema());
+      if (hasUnmodifiedTableSchema()) {
+        result = result && getUnmodifiedTableSchema()
+            .equals(other.getUnmodifiedTableSchema());
+      }
+      result = result &&
+          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();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code AddColumnFamilyStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getUserInfoFieldBuilder();
+          getTableNameFieldBuilder();
+          getColumnfamilySchemaFieldBuilder();
+          getUnmodifiedTableSchemaFieldBuilder();
+        }
+      }
+      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);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_AddColumnFamilyStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData result = new org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (userInfoBuilder_ == null) {
+          result.userInfo_ = userInfo_;
+        } else {
+          result.userInfo_ = userInfoBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (columnfamilySchemaBuilder_ == null) {
+          result.columnfamilySchema_ = columnfamilySchema_;
+        } else {
+          result.columnfamilySchema_ = columnfamilySchemaBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchema_;
+        } else {
+          result.unmodifiedTableSchema_ = unmodifiedTableSchemaBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData.getDefaultInstance()) return this;
+        if (other.hasUserInfo()) {
+          mergeUserInfo(other.getUserInfo());
+        }
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasColumnfamilySchema()) {
+          mergeColumnfamilySchema(other.getColumnfamilySchema());
+        }
+        if (other.hasUnmodifiedTableSchema()) {
+          mergeUnmodifiedTableSchema(other.getUnmodifiedTableSchema());
+        }
+        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;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.AddColumnFamilyStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .UserInformation user_info = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> userInfoBuilder_;
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public boolean hasUserInfo() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo() {
+        if (userInfoBuilder_ == null) {
+          return userInfo_;
+        } else {
+          return userInfoBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          userInfo_ = value;
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder setUserInfo(
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder builderForValue) {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = builderForValue.build();
+          onChanged();
+        } else {
+          userInfoBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder mergeUserInfo(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation value) {
+        if (userInfoBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              userInfo_ != org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance()) {
+            userInfo_ =
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.newBuilder(userInfo_).mergeFrom(value).buildPartial();
+          } else {
+            userInfo_ = value;
+          }
+          onChanged();
+        } else {
+          userInfoBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public Builder clearUserInfo() {
+        if (userInfoBuilder_ == null) {
+          userInfo_ = org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.getDefaultInstance();
+          onChanged();
+        } else {
+          userInfoBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder getUserInfoBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getUserInfoFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder() {
+        if (userInfoBuilder_ != null) {
+          return userInfoBuilder_.getMessageOrBuilder();
+        } else {
+          return userInfo_;
+        }
+      }
+      /**
+       * <code>required .UserInformation user_info = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder> 
+          getUserInfoFieldBuilder() {
+        if (userInfoBuilder_ == null) {
+          userInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder, org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder>(
+                  userInfo_,
+                  getParentForChildren(),
+                  isClean());
+          userInfo_ = null;
+        }
+        return userInfoBuilder_;
+      }
+
+      // required .TableName table_name = 2;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName table_name = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // required .ColumnFamilySchema columnfamily_schema = 3;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> columnfamilySchemaBuilder_;
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public boolean hasColumnfamilySchema() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema() {
+        if (columnfamilySchemaBuilder_ == null) {
+          return columnfamilySchema_;
+        } else {
+          return columnfamilySchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder setColumnfamilySchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema value) {
+        if (columnfamilySchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          columnfamilySchema_ = value;
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder setColumnfamilySchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder builderForValue) {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder mergeColumnfamilySchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema value) {
+        if (columnfamilySchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              columnfamilySchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance()) {
+            columnfamilySchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.newBuilder(columnfamilySchema_).mergeFrom(value).buildPartial();
+          } else {
+            columnfamilySchema_ = value;
+          }
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public Builder clearColumnfamilySchema() {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.getDefaultInstance();
+          onChanged();
+        } else {
+          columnfamilySchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder getColumnfamilySchemaBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getColumnfamilySchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder() {
+        if (columnfamilySchemaBuilder_ != null) {
+          return columnfamilySchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return columnfamilySchema_;
+        }
+      }
+      /**
+       * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder> 
+          getColumnfamilySchemaFieldBuilder() {
+        if (columnfamilySchemaBuilder_ == null) {
+          columnfamilySchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder>(
+                  columnfamilySchema_,
+                  getParentForChildren(),
+                  isClean());
+          columnfamilySchema_ = null;
+        }
+        return columnfamilySchemaBuilder_;
+      }
+
+      // optional .TableSchema unmodified_table_schema = 4;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> unmodifiedTableSchemaBuilder_;
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public boolean hasUnmodifiedTableSchema() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          return unmodifiedTableSchema_;
+        } else {
+          return unmodifiedTableSchemaBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder setUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          unmodifiedTableSchema_ = value;
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder setUnmodifiedTableSchema(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder builderForValue) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = builderForValue.build();
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder mergeUnmodifiedTableSchema(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema value) {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              unmodifiedTableSchema_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance()) {
+            unmodifiedTableSchema_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.newBuilder(unmodifiedTableSchema_).mergeFrom(value).buildPartial();
+          } else {
+            unmodifiedTableSchema_ = value;
+          }
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public Builder clearUnmodifiedTableSchema() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchema_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.getDefaultInstance();
+          onChanged();
+        } else {
+          unmodifiedTableSchemaBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder getUnmodifiedTableSchemaBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getUnmodifiedTableSchemaFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder() {
+        if (unmodifiedTableSchemaBuilder_ != null) {
+          return unmodifiedTableSchemaBuilder_.getMessageOrBuilder();
+        } else {
+          return unmodifiedTableSchema_;
+        }
+      }
+      /**
+       * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder> 
+          getUnmodifiedTableSchemaFieldBuilder() {
+        if (unmodifiedTableSchemaBuilder_ == null) {
+          unmodifiedTableSchemaBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder>(
+                  unmodifiedTableSchema_,
+                  getParentForChildren(),
+                  isClean());
+          unmodifiedTableSchema_ = null;
+        }
+        return unmodifiedTableSchemaBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:AddColumnFamilyStateData)
+    }
+
+    static {
+      defaultInstance = new AddColumnFamilyStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:AddColumnFamilyStateData)
+  }
+
+  public interface ModifyColumnFamilyStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .UserInformation user_info = 1;
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    boolean hasUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation getUserInfo();
+    /**
+     * <code>required .UserInformation user_info = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformationOrBuilder getUserInfoOrBuilder();
+
+    // required .TableName table_name = 2;
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName table_name = 2;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // required .ColumnFamilySchema columnfamily_schema = 3;
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    boolean hasColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema getColumnfamilySchema();
+    /**
+     * <code>required .ColumnFamilySchema columnfamily_schema = 3;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchemaOrBuilder getColumnfamilySchemaOrBuilder();
+
+    // optional .TableSchema unmodified_table_schema = 4;
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    boolean hasUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema getUnmodifiedTableSchema();
+    /**
+     * <code>optional .TableSchema unmodified_table_schema = 4;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchemaOrBuilder getUnmodifiedTableSchemaOrBuilder();
+  }
+  /**
+   * Protobuf type {@code ModifyColumnFamilyStateData}
+   */
+  public static final class ModifyColumnFamilyStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements ModifyColumnFamilyStateDataOrBuilder {
+    // Use ModifyColumnFamilyStateData.newBuilder() to construct.
+    private ModifyColumnFamilyStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ModifyColumnFamilyStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ModifyColumnFamilyStateData defaultInstance;
+    public static ModifyColumnFamilyStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ModifyColumnFamilyStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ModifyColumnFamilyStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = userInfo_.toBuilder();
+              }
+              userInfo_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(userInfo_);
+                userInfo_ = subBuilder.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;
+            }
+          }
+        }
+      } 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_ModifyColumnFamilyStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.internal_static_ModifyColumnFamilyStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData.class, org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ModifyColumnFamilyStateData> PARSER =
+        new com.google.protobuf.AbstractParser<ModifyColumnFamilyStateData>() {
+      public ModifyColumnFamilyStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ModifyColumnFamilyStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ModifyColumnFamilyStateData> 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_;
+    }
+
+    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();
+    }
+    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;
+        }
+      }
+      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_);
+      }
+      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_);
+      }
+      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.ModifyColumnFamilyStateData)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData other = (org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData) 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 &&
+          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();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData 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.ModifyColumnFamilyStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData 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.ModifyColumnFamilyStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData 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.ModifyColumnFamilyStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.

<TRUNCATED>