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/28 15:10:23 UTC

hbase git commit: HBASE-13572 Procedure v2 - client truncate table sync

Repository: hbase
Updated Branches:
  refs/heads/master 75bb815d9 -> 84dd84fc5


HBASE-13572 Procedure v2 - client truncate table sync

Signed-off-by: Matteo Bertozzi <ma...@cloudera.com>


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

Branch: refs/heads/master
Commit: 84dd84fc581d2984daf5e523276b76181c1de5e7
Parents: 75bb815
Author: Ashish Singhi <as...@huawei.com>
Authored: Mon Apr 27 21:21:05 2015 +0530
Committer: Matteo Bertozzi <ma...@cloudera.com>
Committed: Tue Apr 28 12:46:06 2015 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  14 +
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 511 +++++++++++--------
 .../hbase/client/TestProcedureFuture.java       |  17 +-
 .../hbase/protobuf/generated/MasterProtos.java  | 491 +++++++++++-------
 hbase-protocol/src/main/protobuf/Master.proto   |   1 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   3 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   7 +-
 .../hadoop/hbase/master/MasterServices.java     |   2 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   5 +-
 9 files changed, 625 insertions(+), 426 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 a79f666..116a369 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
@@ -318,6 +318,20 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Truncate 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
+   * @param preserveSplits true if the splits should be preserved
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
+   *         operation to complete.
+   */
+  Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
+      throws IOException;
+
+  /**
    * Enable a table.  May timeout.  Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)}
    * and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in
    * disabled state for it to be enabled.

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 11fcff5..28df05c 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
@@ -137,6 +137,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
@@ -622,115 +623,33 @@ public class HBaseAdmin implements Admin {
     return new CreateTableFuture(this, desc, splitKeys, response);
   }
 
-  private static class CreateTableFuture extends ProcedureFuture<Void> {
+  private static class CreateTableFuture extends TableFuture<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);
+      super(admin, desc.getTableName(),
+              (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;
+    protected HTableDescriptor getTableDescriptor() {
+      return desc;
     }
 
     @Override
-    protected Void postOperationResult(final Void result, final long deadlineTs)
-        throws IOException, TimeoutException {
-      LOG.info("Created " + desc.getTableName());
-      return result;
+    protected String getDescription() {
+      return "Creating " + desc.getNameAsString();
     }
 
-    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.");
+    @Override
+    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
+      waitForTableEnabled(deadlineTs);
+      waitForAllRegionsOnline(deadlineTs, splitKeys);
+      return null;
     }
   }
 
@@ -792,13 +711,16 @@ public class HBaseAdmin implements Admin {
     return new DeleteTableFuture(this, tableName, response);
   }
 
-  private static class DeleteTableFuture extends ProcedureFuture<Void> {
-    private final TableName tableName;
-
+  private static class DeleteTableFuture extends TableFuture<Void> {
     public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
         final DeleteTableResponse response) {
-      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
-      this.tableName = tableName;
+      super(admin, tableName,
+              (response != null && response.hasProcId()) ? response.getProcId() : null);
+    }
+
+    @Override
+    protected String getDescription() {
+      return "Deleting " + getTableName();
     }
 
     @Override
@@ -812,30 +734,8 @@ public class HBaseAdmin implements Admin {
     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;
-    }
-
-    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");
-        }
-      });
+      getAdmin().getConnection().clearRegionCache(getTableName());
+      return super.postOperationResult(result, deadlineTs);
     }
   }
 
@@ -883,9 +783,7 @@ public class HBaseAdmin implements Admin {
   }
 
   /**
-   * Truncate a table.
-   * Synchronous operation.
-   *
+   * Truncate a table. Synchronous operation.
    * @param tableName name of table to truncate
    * @param preserveSplits True if the splits should be preserved
    * @throws IOException if a remote or network exception occurs
@@ -893,18 +791,93 @@ public class HBaseAdmin implements Admin {
   @Override
   public void truncateTable(final TableName tableName, final boolean preserveSplits)
       throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection()) {
-      @Override
-      public Void call(int callTimeout) throws ServiceException {
-        TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
-          tableName, preserveSplits);
-        master.truncateTable(null, req);
-        return null;
+    Future<Void> future = truncateTableAsync(tableName, preserveSplits);
+    try {
+      future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException("Interrupted when waiting for table " + tableName
+          + " to be enabled.");
+    } 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());
       }
-    });
+    }
   }
 
   /**
+   * Truncate 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. Asynchronous
+   * operation.
+   * @param tableName name of table to delete
+   * @param preserveSplits true if the splits should be preserved
+   * @throws IOException if a remote or network exception occurs
+   * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the
+   *         operation to complete.
+   */
+  @Override
+  public Future<Void> truncateTableAsync(final TableName tableName, final boolean preserveSplits)
+      throws IOException {
+    TruncateTableResponse response =
+        executeCallable(new MasterCallable<TruncateTableResponse>(getConnection()) {
+          @Override
+          public TruncateTableResponse call(int callTimeout) throws ServiceException {
+            LOG.info("Started enable of " + tableName);
+            TruncateTableRequest req =
+                RequestConverter.buildTruncateTableRequest(tableName, preserveSplits);
+            return master.truncateTable(null, req);
+          }
+        });
+    return new TruncateTableFuture(this, tableName, preserveSplits, response);
+  }
+
+  private static class TruncateTableFuture extends TableFuture<Void> {
+    private final boolean preserveSplits;
+
+    public TruncateTableFuture(final HBaseAdmin admin, final TableName tableName,
+        final boolean preserveSplits, final TruncateTableResponse response) {
+      super(admin, tableName,
+             (response != null && response.hasProcId()) ? response.getProcId() : null);
+      this.preserveSplits = preserveSplits;
+    }
+
+    @Override
+    public String getDescription() {
+      return "Truncating " + getTableName();
+    }
+
+    @Override
+    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
+      waitForTableEnabled(deadlineTs);
+      // once the table is enabled, we know the operation is done. so we can fetch the splitKeys
+      byte[][] splitKeys = preserveSplits ? getAdmin().getTableSplits(getTableName()) : null;
+      waitForAllRegionsOnline(deadlineTs, splitKeys);
+      return null;
+    }
+  }
+
+  private byte[][] getTableSplits(final TableName tableName) throws IOException {
+    byte[][] splits = null;
+    try (RegionLocator locator = getConnection().getRegionLocator(tableName)) {
+      byte[][] startKeys = locator.getStartKeys();
+      if (startKeys.length == 1) {
+        return splits;
+      }
+      splits = new byte[startKeys.length - 1][];
+      for (int i = 1; i < startKeys.length; i++) {
+        splits[i - 1] = startKeys[i];
+      }
+    }
+    return splits;
+  }
+
+
+  /**
    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
    * and {@link #isTableEnabled(byte[])} instead.
    * The table has to be in disabled state for it to be enabled.
@@ -1024,54 +997,22 @@ public class HBaseAdmin implements Admin {
     return new EnableTableFuture(this, tableName, response);
   }
 
-  private static class EnableTableFuture extends ProcedureFuture<Void> {
-    private final TableName tableName;
-
+  private static class EnableTableFuture extends TableFuture<Void> {
     public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
         final EnableTableResponse response) {
-      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
-      this.tableName = tableName;
+      super(admin, tableName,
+              (response != null && response.hasProcId()) ? response.getProcId() : null);
     }
 
     @Override
-    protected Void waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitTableEnabled(deadlineTs);
-      return null;
+    protected String getDescription() {
+      return "Enabling " + getTableName();
     }
 
     @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");
-        }
-      });
+    protected Void waitOperationResult(final long deadlineTs) throws IOException, TimeoutException {
+      waitForTableEnabled(deadlineTs);
+      return null;
     }
   }
 
@@ -1193,48 +1134,23 @@ public class HBaseAdmin implements Admin {
     return new DisableTableFuture(this, tableName, response);
   }
 
-  private static class DisableTableFuture extends ProcedureFuture<Void> {
-    private final TableName tableName;
-
+  private static class DisableTableFuture extends TableFuture<Void> {
     public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
         final DisableTableResponse response) {
-      super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
-      this.tableName = tableName;
+      super(admin, tableName,
+              (response != null && response.hasProcId()) ? response.getProcId() : null);
     }
 
     @Override
-    protected Void waitOperationResult(final long deadlineTs)
-        throws IOException, TimeoutException {
-      waitTableDisabled(deadlineTs);
-      return null;
+    protected String getDescription() {
+      return "Disabling " + getTableName();
     }
 
     @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)
+    protected Void waitOperationResult(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");
-        }
-      });
+      waitForTableDisabled(deadlineTs);
+      return null;
     }
   }
 
@@ -4083,7 +3999,7 @@ public class HBaseAdmin implements Admin {
             result = postOperationResult(result, deadlineTs);
             done = true;
           } catch (IOException e) {
-            result = postOpeartionFailure(e, deadlineTs);
+            result = postOperationFailure(e, deadlineTs);
             done = true;
           }
         } catch (IOException e) {
@@ -4199,9 +4115,9 @@ public class HBaseAdmin implements Admin {
     }
 
     /**
-     * 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().
+     * 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
@@ -4220,7 +4136,7 @@ public class HBaseAdmin implements Admin {
      * @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)
+    protected V postOperationFailure(final IOException exception, final long deadlineTs)
         throws IOException, TimeoutException {
       throw exception;
     }
@@ -4258,4 +4174,175 @@ public class HBaseAdmin implements Admin {
       }
     }
   }
+
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  protected static abstract class TableFuture<V> extends ProcedureFuture<V> {
+    private final TableName tableName;
+
+    public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
+      super(admin, procId);
+      this.tableName = tableName;
+    }
+
+    /**
+     * @return the table name
+     */
+    protected TableName getTableName() {
+      return tableName;
+    }
+
+    /**
+     * @return the table descriptor
+     */
+    protected HTableDescriptor getTableDescriptor() throws IOException {
+      return getAdmin().getTableDescriptorByTableName(getTableName());
+    }
+
+    /**
+     * @return a description of the operation
+     */
+    protected abstract String getDescription();
+
+    @Override
+    protected V postOperationResult(final V result, final long deadlineTs)
+        throws IOException, TimeoutException {
+      LOG.info(getDescription() + " completed");
+      return super.postOperationResult(result, deadlineTs);
+    }
+
+    @Override
+    protected V postOperationFailure(final IOException exception, final long deadlineTs)
+        throws IOException, TimeoutException {
+      LOG.info(getDescription() + " failed with " + exception.getMessage());
+      return super.postOperationFailure(exception, deadlineTs);
+    }
+
+    protected void waitForTableEnabled(final long deadlineTs)
+        throws IOException, TimeoutException {
+      waitForState(deadlineTs, new WaitForStateCallable() {
+        @Override
+        public boolean checkState(int tries) throws IOException {
+          try {
+            if (getAdmin().isTableAvailable(tableName)) {
+              return true;
+            }
+          } catch (TableNotFoundException tnfe) {
+            LOG.debug("Table " + tableName.getNameAsString()
+                + " was not enabled, sleeping. tries=" + tries);
+          }
+          return false;
+        }
+
+        @Override
+        public void throwInterruptedException() throws InterruptedIOException {
+          throw new InterruptedIOException("Interrupted when waiting for table "
+              + tableName.getNameAsString() + " to be enabled");
+        }
+
+        @Override
+        public void throwTimeoutException(long elapsedTime) throws TimeoutException {
+          throw new TimeoutException("Table " + tableName.getNameAsString()
+              + " not enabled after " + elapsedTime + "msec");
+        }
+      });
+    }
+
+    protected void waitForTableDisabled(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");
+        }
+      });
+    }
+
+    protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
+        throws IOException, TimeoutException {
+      final HTableDescriptor desc = getTableDescriptor();
+      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;
+      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.");
+    }
+
+    protected 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");
+        }
+      });
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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
index da3ffe9..641f6c4 100644
--- 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
@@ -19,27 +19,24 @@
 
 package org.apache.hadoop.hbase.client;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 
-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.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
 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> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 463f82f..f660bdd 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
@@ -11154,6 +11154,16 @@ public final class MasterProtos {
 
   public interface TruncateTableResponseOrBuilder
       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 TruncateTableResponse}
@@ -11188,6 +11198,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 {
@@ -11205,6 +11216,11 @@ public final class MasterProtos {
               }
               break;
             }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              procId_ = input.readUInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -11244,7 +11260,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() {
@@ -11258,6 +11292,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);
     }
 
@@ -11267,6 +11304,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;
@@ -11290,6 +11331,11 @@ public final class MasterProtos {
       org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) obj;
 
       boolean result = true;
+      result = result && (hasProcId() == other.hasProcId());
+      if (hasProcId()) {
+        result = result && (getProcId()
+            == other.getProcId());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -11303,6 +11349,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;
@@ -11412,6 +11462,8 @@ public final class MasterProtos {
 
       public Builder clear() {
         super.clear();
+        procId_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
 
@@ -11438,6 +11490,13 @@ public final class MasterProtos {
 
       public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse buildPartial() {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse(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;
       }
@@ -11453,6 +11512,9 @@ public final class MasterProtos {
 
       public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse other) {
         if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance()) return this;
+        if (other.hasProcId()) {
+          setProcId(other.getProcId());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -11478,6 +11540,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:TruncateTableResponse)
     }
@@ -53150,202 +53246,203 @@ public final class MasterProtos {
       "Name\"&\n\023DeleteTableResponse\022\017\n\007proc_id\030\001",
       " \001(\004\"T\n\024TruncateTableRequest\022\035\n\ttableNam" +
       "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\"&\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"
+      "\001(\010:\005false\"(\n\025TruncateTableResponse\022\017\n\007p" +
+      "roc_id\030\001 \001(\004\"4\n\022EnableTableRequest\022\036\n\nta" +
+      "ble_name\030\001 \002(\0132\n.TableName\"&\n\023EnableTabl" +
+      "eResponse\022\017\n\007proc_id\030\001 \001(\004\"5\n\023DisableTab" +
+      "leRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableNam" +
+      "e\"\'\n\024DisableTableResponse\022\017\n\007proc_id\030\001 \001" +
+      "(\004\"X\n\022ModifyTableRequest\022\036\n\ntable_name\030\001" +
+      " \002(\0132\n.TableName\022\"\n\014table_schema\030\002 \002(\0132\014",
+      ".TableSchema\"\025\n\023ModifyTableResponse\"K\n\026C" +
+      "reateNamespaceRequest\0221\n\023namespaceDescri" +
+      "ptor\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027Crea" +
+      "teNamespaceResponse\"/\n\026DeleteNamespaceRe" +
+      "quest\022\025\n\rnamespaceName\030\001 \002(\t\"\031\n\027DeleteNa" +
+      "mespaceResponse\"K\n\026ModifyNamespaceReques" +
+      "t\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.Namespa" +
+      "ceDescriptor\"\031\n\027ModifyNamespaceResponse\"" +
+      "6\n\035GetNamespaceDescriptorRequest\022\025\n\rname" +
+      "spaceName\030\001 \002(\t\"S\n\036GetNamespaceDescripto",
+      "rResponse\0221\n\023namespaceDescriptor\030\001 \002(\0132\024" +
+      ".NamespaceDescriptor\"!\n\037ListNamespaceDes" +
+      "criptorsRequest\"U\n ListNamespaceDescript" +
+      "orsResponse\0221\n\023namespaceDescriptor\030\001 \003(\013" +
+      "2\024.NamespaceDescriptor\"?\n&ListTableDescr" +
+      "iptorsByNamespaceRequest\022\025\n\rnamespaceNam" +
+      "e\030\001 \002(\t\"L\n\'ListTableDescriptorsByNamespa" +
+      "ceResponse\022!\n\013tableSchema\030\001 \003(\0132\014.TableS" +
+      "chema\"9\n ListTableNamesByNamespaceReques" +
+      "t\022\025\n\rnamespaceName\030\001 \002(\t\"B\n!ListTableNam",
+      "esByNamespaceResponse\022\035\n\ttableName\030\001 \003(\013" +
+      "2\n.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutd" +
+      "ownResponse\"\023\n\021StopMasterRequest\"\024\n\022Stop" +
+      "MasterResponse\"\020\n\016BalanceRequest\"\'\n\017Bala" +
+      "nceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031Set" +
+      "BalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013sy" +
+      "nchronous\030\002 \001(\010\"8\n\032SetBalancerRunningRes" +
+      "ponse\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030IsB" +
+      "alancerEnabledRequest\",\n\031IsBalancerEnabl" +
+      "edResponse\022\017\n\007enabled\030\001 \002(\010\"\027\n\025RunCatalo",
+      "gScanRequest\"-\n\026RunCatalogScanResponse\022\023" +
+      "\n\013scan_result\030\001 \001(\005\"-\n\033EnableCatalogJani" +
+      "torRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034EnableCata" +
+      "logJanitorResponse\022\022\n\nprev_value\030\001 \001(\010\" " +
+      "\n\036IsCatalogJanitorEnabledRequest\"0\n\037IsCa" +
+      "talogJanitorEnabledResponse\022\r\n\005value\030\001 \002" +
+      "(\010\"9\n\017SnapshotRequest\022&\n\010snapshot\030\001 \002(\0132" +
+      "\024.SnapshotDescription\",\n\020SnapshotRespons" +
+      "e\022\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034GetComple" +
+      "tedSnapshotsRequest\"H\n\035GetCompletedSnaps",
+      "hotsResponse\022\'\n\tsnapshots\030\001 \003(\0132\024.Snapsh" +
+      "otDescription\"?\n\025DeleteSnapshotRequest\022&" +
+      "\n\010snapshot\030\001 \002(\0132\024.SnapshotDescription\"\030" +
+      "\n\026DeleteSnapshotResponse\"@\n\026RestoreSnaps" +
+      "hotRequest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotD" +
+      "escription\"\031\n\027RestoreSnapshotResponse\"?\n" +
+      "\025IsSnapshotDoneRequest\022&\n\010snapshot\030\001 \001(\013" +
+      "2\024.SnapshotDescription\"U\n\026IsSnapshotDone" +
+      "Response\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010snapsho" +
+      "t\030\002 \001(\0132\024.SnapshotDescription\"F\n\034IsResto",
+      "reSnapshotDoneRequest\022&\n\010snapshot\030\001 \001(\0132" +
+      "\024.SnapshotDescription\"4\n\035IsRestoreSnapsh" +
+      "otDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\"=\n\033G" +
+      "etSchemaAlterStatusRequest\022\036\n\ntable_name" +
+      "\030\001 \002(\0132\n.TableName\"T\n\034GetSchemaAlterStat" +
+      "usResponse\022\035\n\025yet_to_update_regions\030\001 \001(" +
+      "\r\022\025\n\rtotal_regions\030\002 \001(\r\"\202\001\n\032GetTableDes" +
+      "criptorsRequest\022\037\n\013table_names\030\001 \003(\0132\n.T" +
+      "ableName\022\r\n\005regex\030\002 \001(\t\022!\n\022include_sys_t" +
+      "ables\030\003 \001(\010:\005false\022\021\n\tnamespace\030\004 \001(\t\"A\n",
+      "\033GetTableDescriptorsResponse\022\"\n\014table_sc" +
+      "hema\030\001 \003(\0132\014.TableSchema\"[\n\024GetTableName" +
+      "sRequest\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_t" +
+      "ables\030\002 \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"8\n" +
+      "\025GetTableNamesResponse\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\025GetTa" +
+      "bleStateResponse\022 \n\013table_state\030\001 \002(\0132\013." +
+      "TableState\"\031\n\027GetClusterStatusRequest\"B\n" +
+      "\030GetClusterStatusResponse\022&\n\016cluster_sta",
+      "tus\030\001 \002(\0132\016.ClusterStatus\"\030\n\026IsMasterRun" +
+      "ningRequest\"4\n\027IsMasterRunningResponse\022\031" +
+      "\n\021is_master_running\030\001 \002(\010\"@\n\024ExecProcedu" +
+      "reRequest\022(\n\tprocedure\030\001 \002(\0132\025.Procedure" +
+      "Description\"F\n\025ExecProcedureResponse\022\030\n\020" +
+      "expected_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 " +
+      "\001(\014\"B\n\026IsProcedureDoneRequest\022(\n\tprocedu" +
+      "re\030\001 \001(\0132\025.ProcedureDescription\"W\n\027IsPro" +
+      "cedureDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022" +
+      "\'\n\010snapshot\030\002 \001(\0132\025.ProcedureDescription",
+      "\",\n\031GetProcedureResultRequest\022\017\n\007proc_id" +
+      "\030\001 \002(\004\"\347\001\n\032GetProcedureResultResponse\0220\n" +
+      "\005state\030\001 \002(\0162!.GetProcedureResultRespons" +
+      "e.State\022\022\n\nstart_time\030\002 \001(\004\022\023\n\013last_upda" +
+      "te\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\022+\n\texception\030\005 " +
+      "\001(\0132\030.ForeignExceptionMessage\"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\017SetQuotaRequest\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\016bypass_globals\030\006 \001(\010\022\"\n\010th" +
+      "rottle\030\007 \001(\0132\020.ThrottleRequest\"\022\n\020SetQuo" +
+      "taResponse\"A\n\037MajorCompactionTimestampRe" +
+      "quest\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"L\n" +
+      "(MajorCompactionTimestampForRegionReques" +
+      "t\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\"@\n M" +
+      "ajorCompactionTimestampResponse\022\034\n\024compa" +
+      "ction_timestamp\030\001 \002(\0032\343\033\n\rMasterService\022" +
+      "S\n\024GetSchemaAlterStatus\022\034.GetSchemaAlter" +
+      "StatusRequest\032\035.GetSchemaAlterStatusResp",
+      "onse\022P\n\023GetTableDescriptors\022\033.GetTableDe" +
+      "scriptorsRequest\032\034.GetTableDescriptorsRe" +
+      "sponse\022>\n\rGetTableNames\022\025.GetTableNamesR" +
+      "equest\032\026.GetTableNamesResponse\022G\n\020GetClu" +
+      "sterStatus\022\030.GetClusterStatusRequest\032\031.G" +
+      "etClusterStatusResponse\022D\n\017IsMasterRunni" +
+      "ng\022\027.IsMasterRunningRequest\032\030.IsMasterRu" +
+      "nningResponse\0222\n\tAddColumn\022\021.AddColumnRe" +
+      "quest\032\022.AddColumnResponse\022;\n\014DeleteColum" +
+      "n\022\024.DeleteColumnRequest\032\025.DeleteColumnRe",
+      "sponse\022;\n\014ModifyColumn\022\024.ModifyColumnReq" +
+      "uest\032\025.ModifyColumnResponse\0225\n\nMoveRegio" +
+      "n\022\022.MoveRegionRequest\032\023.MoveRegionRespon" +
+      "se\022Y\n\026DispatchMergingRegions\022\036.DispatchM" +
+      "ergingRegionsRequest\032\037.DispatchMergingRe" +
+      "gionsResponse\022;\n\014AssignRegion\022\024.AssignRe" +
+      "gionRequest\032\025.AssignRegionResponse\022A\n\016Un" +
+      "assignRegion\022\026.UnassignRegionRequest\032\027.U" +
+      "nassignRegionResponse\022>\n\rOfflineRegion\022\025" +
+      ".OfflineRegionRequest\032\026.OfflineRegionRes",
+      "ponse\0228\n\013DeleteTable\022\023.DeleteTableReques" +
+      "t\032\024.DeleteTableResponse\022>\n\rtruncateTable" +
+      "\022\025.TruncateTableRequest\032\026.TruncateTableR" +
+      "esponse\0228\n\013EnableTable\022\023.EnableTableRequ" +
+      "est\032\024.EnableTableResponse\022;\n\014DisableTabl" +
+      "e\022\024.DisableTableRequest\032\025.DisableTableRe" +
+      "sponse\0228\n\013ModifyTable\022\023.ModifyTableReque" +
+      "st\032\024.ModifyTableResponse\0228\n\013CreateTable\022" +
+      "\023.CreateTableRequest\032\024.CreateTableRespon" +
+      "se\022/\n\010Shutdown\022\020.ShutdownRequest\032\021.Shutd",
+      "ownResponse\0225\n\nStopMaster\022\022.StopMasterRe" +
+      "quest\032\023.StopMasterResponse\022,\n\007Balance\022\017." +
+      "BalanceRequest\032\020.BalanceResponse\022M\n\022SetB" +
+      "alancerRunning\022\032.SetBalancerRunningReque" +
+      "st\032\033.SetBalancerRunningResponse\022J\n\021IsBal" +
+      "ancerEnabled\022\031.IsBalancerEnabledRequest\032" +
+      "\032.IsBalancerEnabledResponse\022A\n\016RunCatalo" +
+      "gScan\022\026.RunCatalogScanRequest\032\027.RunCatal" +
+      "ogScanResponse\022S\n\024EnableCatalogJanitor\022\034" +
+      ".EnableCatalogJanitorRequest\032\035.EnableCat",
+      "alogJanitorResponse\022\\\n\027IsCatalogJanitorE" +
+      "nabled\022\037.IsCatalogJanitorEnabledRequest\032" +
+      " .IsCatalogJanitorEnabledResponse\022L\n\021Exe" +
+      "cMasterService\022\032.CoprocessorServiceReque" +
+      "st\032\033.CoprocessorServiceResponse\022/\n\010Snaps" +
+      "hot\022\020.SnapshotRequest\032\021.SnapshotResponse" +
+      "\022V\n\025GetCompletedSnapshots\022\035.GetCompleted" +
+      "SnapshotsRequest\032\036.GetCompletedSnapshots" +
+      "Response\022A\n\016DeleteSnapshot\022\026.DeleteSnaps" +
+      "hotRequest\032\027.DeleteSnapshotResponse\022A\n\016I",
+      "sSnapshotDone\022\026.IsSnapshotDoneRequest\032\027." +
+      "IsSnapshotDoneResponse\022D\n\017RestoreSnapsho" +
+      "t\022\027.RestoreSnapshotRequest\032\030.RestoreSnap" +
+      "shotResponse\022V\n\025IsRestoreSnapshotDone\022\035." +
+      "IsRestoreSnapshotDoneRequest\032\036.IsRestore" +
+      "SnapshotDoneResponse\022>\n\rExecProcedure\022\025." +
+      "ExecProcedureRequest\032\026.ExecProcedureResp" +
+      "onse\022E\n\024ExecProcedureWithRet\022\025.ExecProce" +
+      "dureRequest\032\026.ExecProcedureResponse\022D\n\017I" +
+      "sProcedureDone\022\027.IsProcedureDoneRequest\032",
+      "\030.IsProcedureDoneResponse\022D\n\017ModifyNames" +
+      "pace\022\027.ModifyNamespaceRequest\032\030.ModifyNa" +
+      "mespaceResponse\022D\n\017CreateNamespace\022\027.Cre" +
+      "ateNamespaceRequest\032\030.CreateNamespaceRes" +
+      "ponse\022D\n\017DeleteNamespace\022\027.DeleteNamespa" +
+      "ceRequest\032\030.DeleteNamespaceResponse\022Y\n\026G" +
+      "etNamespaceDescriptor\022\036.GetNamespaceDesc" +
+      "riptorRequest\032\037.GetNamespaceDescriptorRe" +
+      "sponse\022_\n\030ListNamespaceDescriptors\022 .Lis" +
+      "tNamespaceDescriptorsRequest\032!.ListNames",
+      "paceDescriptorsResponse\022t\n\037ListTableDesc" +
+      "riptorsByNamespace\022\'.ListTableDescriptor" +
+      "sByNamespaceRequest\032(.ListTableDescripto" +
+      "rsByNamespaceResponse\022b\n\031ListTableNamesB" +
+      "yNamespace\022!.ListTableNamesByNamespaceRe" +
+      "quest\032\".ListTableNamesByNamespaceRespons" +
+      "e\022>\n\rGetTableState\022\025.GetTableStateReques" +
+      "t\032\026.GetTableStateResponse\022/\n\010SetQuota\022\020." +
+      "SetQuotaRequest\032\021.SetQuotaResponse\022f\n\037ge" +
+      "tLastMajorCompactionTimestamp\022 .MajorCom",
+      "pactionTimestampRequest\032!.MajorCompactio" +
+      "nTimestampResponse\022x\n(getLastMajorCompac" +
+      "tionTimestampForRegion\022).MajorCompaction" +
+      "TimestampForRegionRequest\032!.MajorCompact" +
+      "ionTimestampResponse\022M\n\022getProcedureResu" +
+      "lt\022\032.GetProcedureResultRequest\032\033.GetProc" +
+      "edureResultResponseBB\n*org.apache.hadoop" +
+      ".hbase.protobuf.generatedB\014MasterProtosH" +
+      "\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -53483,7 +53580,7 @@ public final class MasterProtos {
           internal_static_TruncateTableResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_TruncateTableResponse_descriptor,
-              new java.lang.String[] { });
+              new java.lang.String[] { "ProcId", });
           internal_static_EnableTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(22);
           internal_static_EnableTableRequest_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 d5f4275..ce2b940 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -126,6 +126,7 @@ message TruncateTableRequest {
 }
 
 message TruncateTableResponse {
+  optional uint64 proc_id = 1;
 }
 
 message EnableTableRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 98006ec..33c2761 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
@@ -1605,7 +1605,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   }
 
   @Override
-  public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
+  public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
     checkInitialized();
     if (cpHost != null) {
       cpHost.preTruncateTable(tableName);
@@ -1619,6 +1619,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     if (cpHost != null) {
       cpHost.postTruncateTable(tableName);
     }
+    return procId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 91c406c..589c169 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
@@ -477,12 +477,13 @@ public class MasterRpcServices extends RSRpcServices
   public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
       throws ServiceException {
     try {
-      master.truncateTable(ProtobufUtil.toTableName(request.getTableName()),
-        request.getPreserveSplits());
+      long procId =
+          master.truncateTable(ProtobufUtil.toTableName(request.getTableName()),
+            request.getPreserveSplits());
+      return TruncateTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
     }
-    return TruncateTableResponse.newBuilder().build();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 6153139..b3b1e64 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
@@ -121,7 +121,7 @@ public interface MasterServices extends Server {
    * @param preserveSplits True if the splits should be preserved
    * @throws IOException
    */
-  public void truncateTable(final TableName tableName, boolean preserveSplits) throws IOException;
+  public long truncateTable(final TableName tableName, boolean preserveSplits) throws IOException;
 
   /**
    * Modify the descriptor of an existing table

http://git-wip-us.apache.org/repos/asf/hbase/blob/84dd84fc/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 00cad06..e34db29 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
@@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
@@ -433,7 +432,9 @@ public class TestCatalogJanitor {
     }
 
     @Override
-    public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { }
+    public long truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
+      return -1;
+    }
 
 
     @Override