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 2016/08/30 17:06:35 UTC

[2/2] hbase git commit: HBASE-16519 Procedure v2 - Avoid sync wait on DDLs operation

HBASE-16519 Procedure v2 - Avoid sync wait on DDLs operation


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

Branch: refs/heads/master
Commit: 4a4f8e704903f02dac38d584ab85a472a9f3d2ce
Parents: 2acd788
Author: Matteo Bertozzi <ma...@cloudera.com>
Authored: Tue Aug 30 09:45:56 2016 -0700
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Tue Aug 30 09:45:56 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   | 57 ++++++++++++++++----
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 38 ++++++++++---
 .../org/apache/hadoop/hbase/master/HMaster.java | 31 +++++++----
 .../procedure/AddColumnFamilyProcedure.java     | 19 +++++--
 .../procedure/DeleteColumnFamilyProcedure.java  | 19 +++++--
 .../procedure/ModifyColumnFamilyProcedure.java  | 19 +++++--
 .../master/procedure/ModifyTableProcedure.java  | 20 +++++--
 .../master/procedure/ProcedurePrepareLatch.java | 22 ++++++--
 .../master/procedure/ProcedureSyncWait.java     |  2 +-
 .../procedure/TruncateTableProcedure.java       | 15 ++++++
 10 files changed, 198 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 0610517..321ea55 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -506,6 +506,16 @@ public interface Admin extends Abortable, Closeable {
     throws IOException;
 
   /**
+   * Add a column family to an existing table.
+   *
+   * @param tableName name of the table to add column family to
+   * @param columnFamily column family descriptor of column family to be added
+   * @throws IOException if a remote or network exception occurs
+   */
+  void addColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
+    throws IOException;
+
+  /**
    * Add a column family to an existing table. Asynchronous operation.
    * 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
@@ -518,7 +528,7 @@ public interface Admin extends Abortable, Closeable {
    * @return the result of the async add column family. You can use Future.get(long, TimeUnit) to
    *         wait on the operation to complete.
    */
-  Future<Void> addColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
+  Future<Void> addColumnFamilyAsync(final TableName tableName, final HColumnDescriptor columnFamily)
       throws IOException;
 
   /**
@@ -537,6 +547,15 @@ public interface Admin extends Abortable, Closeable {
 
   /**
    * Delete a column family from a table. Asynchronous operation.
+   *
+   * @param tableName name of table
+   * @param columnFamily name of column family to be deleted
+   * @throws IOException if a remote or network exception occurs
+   */
+  void deleteColumnFamily(final TableName tableName, final byte[] columnFamily) throws IOException;
+
+  /**
+   * Delete a column family from a table. Asynchronous operation.
    * 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
@@ -548,15 +567,11 @@ public interface Admin extends Abortable, Closeable {
    * @return the result of the async delete column family. You can use Future.get(long, TimeUnit) to
    *         wait on the operation to complete.
    */
-  Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
+  Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
       throws IOException;
 
   /**
-   * Modify an existing column family on a table. Asynchronous operation.
-   * 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.
+   * Modify an existing column family on a table.
    *
    * @param tableName name of table
    * @param columnFamily new column family descriptor to use
@@ -571,7 +586,21 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Modify an existing column family on a table.
+   *
+   * @param tableName name of table
+   * @param columnFamily new column family descriptor to use
+   * @throws IOException if a remote or network exception occurs
+   */
+  void modifyColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
+      throws IOException;
+
+  /**
    * Modify an existing column family on a table. Asynchronous operation.
+   * 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
    * @param columnFamily new column family descriptor to use
@@ -579,7 +608,7 @@ public interface Admin extends Abortable, Closeable {
    * @return the result of the async modify column family. You can use Future.get(long, TimeUnit) to
    *         wait on the operation to complete.
    */
-  Future<Void> modifyColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
+  Future<Void> modifyColumnFamilyAsync(TableName tableName, HColumnDescriptor columnFamily)
       throws IOException;
 
 
@@ -927,6 +956,16 @@ public interface Admin extends Abortable, Closeable {
     throws IOException;
 
   /**
+   * Modify an existing table, more IRB friendly version.
+   *
+   * @param tableName name of table.
+   * @param htd modified description of the table
+   * @throws IOException if a remote or network exception occurs
+   */
+  void modifyTable(final TableName tableName, final HTableDescriptor htd)
+      throws IOException;
+
+  /**
    * Modify an existing table, more IRB friendly version. Asynchronous operation.  This means that
    * it may be a while before your schema change is updated across all of the table.
    * You can use Future.get(long, TimeUnit) to wait on the operation to complete.
@@ -940,7 +979,7 @@ public interface Admin extends Abortable, Closeable {
    * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the
    *     operation to complete
    */
-  Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
+  Future<Void> modifyTableAsync(final TableName tableName, final HTableDescriptor htd)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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 48a614f..6035895 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
@@ -863,7 +863,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> addColumnFamily(final TableName tableName,
+  public void addColumnFamily(final TableName tableName, final HColumnDescriptor columnFamily)
+      throws IOException {
+    get(addColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> addColumnFamilyAsync(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     AddColumnResponse response =
         executeCallable(new MasterCallable<AddColumnResponse>(getConnection(),
@@ -906,7 +912,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
+  public void deleteColumnFamily(final TableName tableName, final byte[] columnFamily)
+      throws IOException {
+    get(deleteColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> deleteColumnFamilyAsync(final TableName tableName, final byte[] columnFamily)
       throws IOException {
     DeleteColumnResponse response =
         executeCallable(new MasterCallable<DeleteColumnResponse>(getConnection(),
@@ -917,8 +929,7 @@ public class HBaseAdmin implements Admin {
             DeleteColumnRequest req =
                 RequestConverter.buildDeleteColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.deleteColumn(getRpcController(), req);
-            return null;
+            return master.deleteColumn(getRpcController(), req);
           }
         });
     return new DeleteColumnFamilyFuture(this, tableName, response);
@@ -950,7 +961,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> modifyColumnFamily(final TableName tableName,
+  public void modifyColumnFamily(final TableName tableName,
+      final HColumnDescriptor columnFamily) throws IOException {
+    get(modifyColumnFamilyAsync(tableName, columnFamily), syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> modifyColumnFamilyAsync(final TableName tableName,
       final HColumnDescriptor columnFamily) throws IOException {
     ModifyColumnResponse response =
         executeCallable(new MasterCallable<ModifyColumnResponse>(getConnection(),
@@ -961,8 +978,7 @@ public class HBaseAdmin implements Admin {
             ModifyColumnRequest req =
                 RequestConverter.buildModifyColumnRequest(tableName, columnFamily,
                   ng.getNonceGroup(), ng.newNonce());
-            master.modifyColumn(getRpcController(), req);
-            return null;
+            return master.modifyColumn(getRpcController(), req);
           }
         });
     return new ModifyColumnFamilyFuture(this, tableName, response);
@@ -1642,7 +1658,13 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
-  public Future<Void> modifyTable(final TableName tableName, final HTableDescriptor htd)
+  public void modifyTable(final TableName tableName, final HTableDescriptor htd)
+      throws IOException {
+    get(modifyTableAsync(tableName, htd), syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> modifyTableAsync(final TableName tableName, final HTableDescriptor htd)
       throws IOException {
     if (!tableName.equals(htd.getTableName())) {
       throw new IllegalArgumentException("the specified table name '" + tableName +

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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 f4c2c1c..c643fa8 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
@@ -1847,11 +1847,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
 
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     long procId = this.procedureExecutor.submitProcedure(
-      new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits),
+      new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName,
+        preserveSplits, latch),
       nonceGroup,
       nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    latch.await();
 
     if (cpHost != null) {
       cpHost.postTruncateTable(tableName);
@@ -1876,11 +1878,14 @@ public class HMaster extends HRegionServer implements MasterServices {
       }
     }
     // Execute the operation synchronously - wait for the operation to complete before continuing.
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     long procId = this.procedureExecutor.submitProcedure(
-      new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnDescriptor),
+      new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
+        columnDescriptor, latch),
       nonceGroup,
       nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    latch.await();
+
     if (cpHost != null) {
       cpHost.postAddColumn(tableName, columnDescriptor);
     }
@@ -1906,11 +1911,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
 
     // Execute the operation synchronously - wait for the operation to complete before continuing.
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     long procId = this.procedureExecutor.submitProcedure(
-      new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, descriptor),
+      new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
+        descriptor, latch),
       nonceGroup,
       nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    latch.await();
 
     if (cpHost != null) {
       cpHost.postModifyColumn(tableName, descriptor);
@@ -1934,11 +1941,13 @@ public class HMaster extends HRegionServer implements MasterServices {
     LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
 
     // Execute the operation synchronously - wait for the operation to complete before continuing.
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     long procId = this.procedureExecutor.submitProcedure(
-      new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName, columnName),
+      new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(), tableName,
+        columnName, latch),
       nonceGroup,
       nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    latch.await();
 
     if (cpHost != null) {
       cpHost.postDeleteColumn(tableName, columnName);
@@ -2060,12 +2069,12 @@ public class HMaster extends HRegionServer implements MasterServices {
     LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
 
     // Execute the operation synchronously - wait for the operation completes before continuing.
+    ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
     long procId = this.procedureExecutor.submitProcedure(
-      new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor),
+      new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor, latch),
       nonceGroup,
       nonce);
-
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    latch.await();
 
     if (cpHost != null) {
       cpHost.postModifyTable(tableName, descriptor);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index 195f738..c9478cb 100644
--- 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
@@ -59,16 +59,23 @@ public class AddColumnFamilyProcedure
   private List<HRegionInfo> regionInfoList;
   private Boolean traceEnabled;
 
+  // used for compatibility with old clients, until 2.0 the client had a sync behavior
+  private final ProcedurePrepareLatch syncLatch;
+
   public AddColumnFamilyProcedure() {
     this.unmodifiedHTableDescriptor = null;
     this.regionInfoList = null;
     this.traceEnabled = null;
+    this.syncLatch = null;
   }
 
-  public AddColumnFamilyProcedure(
-      final MasterProcedureEnv env,
-      final TableName tableName,
+  public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
       final HColumnDescriptor cfDescriptor) throws IOException {
+    this(env, tableName, cfDescriptor, null);
+  }
+
+  public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
+      final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) throws IOException {
     this.tableName = tableName;
     this.cfDescriptor = cfDescriptor;
     this.user = env.getRequestUser();
@@ -76,6 +83,7 @@ public class AddColumnFamilyProcedure
     this.unmodifiedHTableDescriptor = null;
     this.regionInfoList = null;
     this.traceEnabled = null;
+    this.syncLatch = latch;
   }
 
   @Override
@@ -152,6 +160,11 @@ public class AddColumnFamilyProcedure
   }
 
   @Override
+  protected void completionCleanup(final MasterProcedureEnv env) {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
+
+  @Override
   protected AddColumnFamilyState getState(final int stateId) {
     return AddColumnFamilyState.valueOf(stateId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index 8bcbd82..da24cef 100644
--- 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
@@ -62,16 +62,23 @@ public class DeleteColumnFamilyProcedure
   private List<HRegionInfo> regionInfoList;
   private Boolean traceEnabled;
 
+  // used for compatibility with old clients, until 2.0 the client had a sync behavior
+  private final ProcedurePrepareLatch syncLatch;
+
   public DeleteColumnFamilyProcedure() {
     this.unmodifiedHTableDescriptor = null;
     this.regionInfoList = null;
     this.traceEnabled = null;
+    this.syncLatch = null;
   }
 
-  public DeleteColumnFamilyProcedure(
-      final MasterProcedureEnv env,
-      final TableName tableName,
+  public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
       final byte[] familyName) throws IOException {
+    this(env, tableName, familyName, null);
+  }
+
+  public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
+      final byte[] familyName, final ProcedurePrepareLatch latch) throws IOException {
     this.tableName = tableName;
     this.familyName = familyName;
     this.user = env.getRequestUser();
@@ -79,6 +86,7 @@ public class DeleteColumnFamilyProcedure
     this.unmodifiedHTableDescriptor = null;
     this.regionInfoList = null;
     this.traceEnabled = null;
+    this.syncLatch = latch;
   }
 
   @Override
@@ -170,6 +178,11 @@ public class DeleteColumnFamilyProcedure
   }
 
   @Override
+  protected void completionCleanup(final MasterProcedureEnv env) {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
+
+  @Override
   protected DeleteColumnFamilyState getState(final int stateId) {
     return DeleteColumnFamilyState.valueOf(stateId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index 6a408da..1769306 100644
--- 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
@@ -59,21 +59,29 @@ public class ModifyColumnFamilyProcedure
 
   private Boolean traceEnabled;
 
+  // used for compatibility with old clients, until 2.0 the client had a sync behavior
+  private final ProcedurePrepareLatch syncLatch;
+
   public ModifyColumnFamilyProcedure() {
     this.unmodifiedHTableDescriptor = null;
     this.traceEnabled = null;
+    this.syncLatch = null;
   }
 
-  public ModifyColumnFamilyProcedure(
-      final MasterProcedureEnv env,
-      final TableName tableName,
+  public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
       final HColumnDescriptor cfDescriptor) throws IOException {
+    this(env, tableName, cfDescriptor, null);
+  }
+
+  public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
+      final HColumnDescriptor cfDescriptor, final ProcedurePrepareLatch latch) throws IOException {
     this.tableName = tableName;
     this.cfDescriptor = cfDescriptor;
     this.user = env.getRequestUser();
     this.setOwner(this.user.getShortName());
     this.unmodifiedHTableDescriptor = null;
     this.traceEnabled = null;
+    this.syncLatch = latch;
   }
 
   @Override
@@ -150,6 +158,11 @@ public class ModifyColumnFamilyProcedure
   }
 
   @Override
+  protected void completionCleanup(final MasterProcedureEnv env) {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
+
+  @Override
   protected ModifyColumnFamilyState getState(final int stateId) {
     return ModifyColumnFamilyState.valueOf(stateId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index c523f23..8299bcc 100644
--- 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
@@ -68,17 +68,26 @@ public class ModifyTableProcedure
   private List<HRegionInfo> regionInfoList;
   private Boolean traceEnabled = null;
 
+  // used for compatibility with old clients, until 2.0 the client had a sync behavior
+  private final ProcedurePrepareLatch syncLatch;
+
   public ModifyTableProcedure() {
     initilize();
+    this.syncLatch = null;
   }
 
-  public ModifyTableProcedure(
-    final MasterProcedureEnv env,
-    final HTableDescriptor htd) throws IOException {
+  public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd)
+      throws IOException {
+    this(env, htd, null);
+  }
+
+  public ModifyTableProcedure(final MasterProcedureEnv env, final HTableDescriptor htd,
+      final ProcedurePrepareLatch latch) throws IOException {
     initilize();
     this.modifiedHTableDescriptor = htd;
     this.user = env.getRequestUser();
     this.setOwner(this.user.getShortName());
+    this.syncLatch = latch;
   }
 
   private void initilize() {
@@ -185,6 +194,11 @@ public class ModifyTableProcedure
   }
 
   @Override
+  protected void completionCleanup(final MasterProcedureEnv env) {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
+
+  @Override
   protected ModifyTableState getState(final int stateId) {
     return ModifyTableState.valueOf(stateId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index b13e44d..eaeb9ac 100644
--- 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
@@ -36,13 +36,29 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 public abstract class ProcedurePrepareLatch {
   private static final NoopLatch noopLatch = new NoopLatch();
 
+  /**
+   * Create a latch if the client does not have async proc support.
+   * This uses the default 1.1 version.
+   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
+   */
   public static ProcedurePrepareLatch createLatch() {
+    // don't use the latch if we have procedure support (default 1.1)
+    return createLatch(1, 1);
+  }
+
+  /**
+   * Create a latch if the client does not have async proc support
+   * @param major major version with async proc support
+   * @param minor minor version with async proc support
+   * @return a CompatibilityLatch or a NoopLatch if the client has async proc support
+   */
+  public static ProcedurePrepareLatch createLatch(int major, int minor) {
     // don't use the latch if we have procedure support
-    return hasProcedureSupport() ? noopLatch : new CompatibilityLatch();
+    return hasProcedureSupport(major, minor) ? noopLatch : new CompatibilityLatch();
   }
 
-  public static boolean hasProcedureSupport() {
-    return VersionInfoUtil.currentClientHasMinimumVersion(1, 1);
+  private static boolean hasProcedureSupport(int major, int minor) {
+    return VersionInfoUtil.currentClientHasMinimumVersion(major, minor);
   }
 
   protected abstract void countDown(final Procedure proc);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/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
index 8a5eb35..cf8fdd4 100644
--- 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
@@ -71,7 +71,7 @@ public final class ProcedureSyncWait {
     return waitForProcedureToComplete(procExec, procId);
   }
 
-  public static byte[] waitForProcedureToComplete(ProcedureExecutor<MasterProcedureEnv> procExec,
+  private 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

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a4f8e70/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
index 0b60cea..8e3ef3e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java
@@ -56,16 +56,26 @@ public class TruncateTableProcedure
   private HTableDescriptor hTableDescriptor;
   private TableName tableName;
 
+  // used for compatibility with old clients, until 2.0 the client had a sync behavior
+  private final ProcedurePrepareLatch syncLatch;
+
   public TruncateTableProcedure() {
     // Required by the Procedure framework to create the procedure on replay
+    syncLatch = null;
   }
 
   public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
       boolean preserveSplits) throws IOException {
+    this(env, tableName, preserveSplits, null);
+  }
+
+  public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
+      boolean preserveSplits, ProcedurePrepareLatch latch) throws IOException {
     this.tableName = tableName;
     this.preserveSplits = preserveSplits;
     this.user = env.getRequestUser();
     this.setOwner(this.user.getShortName());
+    this.syncLatch = latch;
   }
 
   @Override
@@ -150,6 +160,11 @@ public class TruncateTableProcedure
   }
 
   @Override
+  protected void completionCleanup(final MasterProcedureEnv env) {
+    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
+  }
+
+  @Override
   protected TruncateTableState getState(final int stateId) {
     return TruncateTableState.valueOf(stateId);
   }