You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/10/25 02:57:33 UTC

[1/2] HBASE-12142 Truncate command does not preserve ACLs table (Vandana Ayyalasomayajula)

Repository: hbase
Updated Branches:
  refs/heads/0.98 714e8368c -> 5b1380c3c


http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
new file mode 100644
index 0000000..d13bca5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
@@ -0,0 +1,247 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.handler;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Truncate the table by removing META and the HDFS files and recreating it.
+ * If the 'preserveSplits' option is set to true, the region splits are preserved on recreate.
+ *
+ * If the operation fails in the middle it may require hbck to fix the system state.
+ */
+@InterfaceAudience.Private
+public class TruncateTableHandler extends DeleteTableHandler {
+  private static final Log LOG = LogFactory.getLog(TruncateTableHandler.class);
+
+  private final boolean preserveSplits;
+
+  public TruncateTableHandler(final TableName tableName, final Server server,
+      final MasterServices masterServices, boolean preserveSplits) {
+    super(tableName, server, masterServices);
+    this.preserveSplits = preserveSplits;
+  }
+
+  @Override
+  protected void handleTableOperation(List<HRegionInfo> regions)
+      throws IOException, KeeperException {
+    MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preTruncateTableHandler(this.tableName);
+    }
+
+    // 1. Wait because of region in transition
+    waitRegionInTransition(regions);
+
+    // 2. Remove table from .META. and HDFS
+    removeTableData(regions);
+
+    // -----------------------------------------------------------------------
+    // PONR: At this point the table is deleted.
+    //       If the recreate fails, the user can only re-create the table.
+    // -----------------------------------------------------------------------
+
+    // 3. Recreate the regions
+    recreateTable(regions);
+
+    if (cpHost != null) {
+      cpHost.postTruncateTableHandler(this.tableName);
+    }
+  }
+
+  private void recreateTable(final List<HRegionInfo> regions) throws IOException {
+    MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
+    Path tempdir = mfs.getTempDir();
+    FileSystem fs = mfs.getFileSystem();
+
+    AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
+
+    // 1. Set table znode
+    checkAndSetEnablingTable(assignmentManager, tableName);
+    try {
+      // 1. Create Table Descriptor
+      new FSTableDescriptors(server.getConfiguration())
+        .createTableDescriptorForTableDirectory(tempdir, getTableDescriptor(), false);
+      Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
+      Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
+
+      HRegionInfo[] newRegions;
+      if (this.preserveSplits) {
+        newRegions = regions.toArray(new HRegionInfo[regions.size()]);
+        LOG.info("Truncate will preserve " + newRegions.length + " regions");
+      } else {
+        newRegions = new HRegionInfo[1];
+        newRegions[0] = new HRegionInfo(this.tableName, null, null);
+        LOG.info("Truncate will not preserve the regions");
+      }
+
+      // 2. Create Regions
+      List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
+        masterServices.getConfiguration(), tempdir,
+        getTableDescriptor(), newRegions, null);
+
+      // 3. Move Table temp directory to the hbase root location
+      if (!fs.rename(tempTableDir, tableDir)) {
+        throw new IOException("Unable to move table from temp=" + tempTableDir +
+          " to hbase root=" + tableDir);
+      }
+
+      // 4. Add regions to META
+      MetaEditor.addRegionsToMeta(masterServices.getCatalogTracker(), regionInfos);
+
+      // 5. Trigger immediate assignment of the regions in round-robin fashion
+      ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
+
+      // 6. Set table enabled flag up in zk.
+      try {
+        assignmentManager.getZKTable().setEnabledTable(tableName);
+      } catch (KeeperException e) {
+        throw new IOException("Unable to ensure that " + tableName + " will be" +
+          " enabled because of a ZooKeeper issue", e);
+      }
+    } catch (IOException e) {
+      removeEnablingTable(assignmentManager, tableName);
+      throw e;
+    }
+  }
+
+  void checkAndSetEnablingTable(final AssignmentManager assignmentManager, final TableName tableName)
+      throws IOException {
+    // If we have multiple client threads trying to create the table at the
+    // same time, given the async nature of the operation, the table
+    // could be in a state where hbase:meta table hasn't been updated yet in
+    // the process() function.
+    // Use enabling state to tell if there is already a request for the same
+    // table in progress. This will introduce a new zookeeper call. Given
+    // createTable isn't a frequent operation, that should be ok.
+    // TODO: now that we have table locks, re-evaluate above -- table locks are not enough.
+    // We could have cleared the hbase.rootdir and not zk. How can we detect this case?
+    // Having to clean zk AND hdfs is awkward.
+    try {
+      if (!assignmentManager.getZKTable().checkAndSetEnablingTable(tableName)) {
+        throw new TableExistsException(tableName);
+      }
+    } catch (KeeperException e) {
+      throw new IOException("Unable to ensure that the table will be"
+          + " enabling because of a ZooKeeper issue", e);
+    }
+  }
+
+  void removeEnablingTable(final AssignmentManager assignmentManager, final TableName tableName) {
+    // Try deleting the enabling node in case of error
+    // If this does not happen then if the client tries to create the table
+    // again with the same Active master
+    // It will block the creation saying TableAlreadyExists.
+    try {
+      assignmentManager.getZKTable().removeEnablingTable(tableName, false);
+    } catch (KeeperException e) {
+      // Keeper exception should not happen here
+      LOG.error("Got a keeper exception while removing the ENABLING table znode " + tableName, e);
+    }
+  }
+
+  /**
+   * Removes the table from .META. and archives the HDFS files.
+   */
+  void removeTableData(final List<HRegionInfo> regions) throws IOException, KeeperException {
+    // 1. Remove regions from META
+    LOG.debug("Deleting regions from META");
+    MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
+
+    // -----------------------------------------------------------------------
+    // NOTE: At this point we still have data on disk, but nothing in .META.
+    // if the rename below fails, hbck will report an inconsistency.
+    // -----------------------------------------------------------------------
+
+    // 2. Move the table in /hbase/.tmp
+    MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
+    Path tempTableDir = mfs.moveTableToTemp(tableName);
+
+    // 3. Archive regions from FS (temp directory)
+    FileSystem fs = mfs.getFileSystem();
+    for (HRegionInfo hri : regions) {
+      LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
+      HFileArchiver.archiveRegion(fs, mfs.getRootDir(), tempTableDir,
+        HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
+    }
+
+    // 4. Delete table directory from FS (temp directory)
+    if (!fs.delete(tempTableDir, true)) {
+      LOG.error("Couldn't delete " + tempTableDir);
+    }
+
+    LOG.debug("Table '" + tableName + "' archived!");
+  }
+
+  void waitRegionInTransition(final List<HRegionInfo> regions) throws IOException {
+    AssignmentManager am = this.masterServices.getAssignmentManager();
+    RegionStates states = am.getRegionStates();
+    long waitTime = server.getConfiguration().getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
+    for (HRegionInfo region : regions) {
+      long done = System.currentTimeMillis() + waitTime;
+      while (System.currentTimeMillis() < done) {
+        if (states.isRegionInState(region, State.FAILED_OPEN)) {
+          am.regionOffline(region);
+        }
+        if (!states.isRegionInTransition(region)) {
+          break;
+        }
+        try {
+          Thread.sleep(waitingTimeForEvents);
+        } catch (InterruptedException e) {
+          LOG.warn("Interrupted while sleeping");
+          throw (InterruptedIOException) new InterruptedIOException().initCause(e);
+        }
+        LOG.debug("Waiting on region to clear regions in transition; "
+            + am.getRegionStates().getRegionTransitionState(region));
+      }
+      if (states.isRegionInTransition(region)) {
+        throw new IOException("Waited hbase.master.wait.on.region (" + waitTime
+            + "ms) for region to leave region " + region.getRegionNameAsString()
+            + " in transitions");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 4e87bbc..40bbbad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -172,6 +172,8 @@ public class AccessController extends BaseMasterAndRegionObserver
   private Map<InternalScanner,String> scannerOwners =
       new MapMaker().weakKeys().makeMap();
 
+  private Map<TableName, List<UserPermission>> tableAcls;
+
   // Provider for mapping principal names to Users
   private UserProvider userProvider;
 
@@ -871,12 +873,44 @@ public class AccessController extends BaseMasterAndRegionObserver
           + " is required for " + AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT
           + " to have an effect");
     }
+
+    tableAcls = new MapMaker().weakValues().makeMap();
   }
 
   public void stop(CoprocessorEnvironment env) {
 
   }
 
+   @Override
+  public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
+      throws IOException {
+    requirePermission("truncateTable", tableName, null, null, Action.ADMIN);
+    List<UserPermission> acls = AccessControlLists.getUserTablePermissions(c.getEnvironment()
+        .getConfiguration(), tableName);
+    if (acls != null) {
+      tableAcls.put(tableName, acls);
+    }
+  }
+
+  @Override
+  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+    List<UserPermission> perms = tableAcls.get(tableName);
+    if (perms != null) {
+      for (UserPermission perm : perms) {
+        AccessControlLists.addUserPermission(ctx.getEnvironment().getConfiguration(), perm);
+      }
+    }
+    tableAcls.remove(tableName);
+  }
+
+  @Override
+  public void preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      TableName tableName) throws IOException {}
+  @Override
+  public void postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+      TableName tableName) throws IOException {}
+
   @Override
   public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
       HTableDescriptor desc, HRegionInfo[] regions) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index f74179b..98cdbba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -835,4 +835,24 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP;
     }
   }
+
+  @Override
+  public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
+
+  @Override
+  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
+
+  @Override
+  public void preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
+
+  @Override
+  public void postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 205eaea..4508835 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.master.AssignmentManager;
 
 /**
  * Utility methods for interacting with the regions.
@@ -251,4 +252,24 @@ public abstract class ModifyRegionUtils {
         });
     return regionOpenAndInitThreadPool;
   }
+
+  /**
+   * Triggers a bulk assignment of the specified regions
+   *
+   * @param assignmentManager the Assignment Manger
+   * @param regionInfos the list of regions to assign
+   * @throws IOException if an error occurred during the assignment
+   */
+  public static void assignRegions(final AssignmentManager assignmentManager,
+      final List<HRegionInfo> regionInfos) throws IOException {
+    try {
+      assignmentManager.getRegionStates().createRegionStates(regionInfos);
+      assignmentManager.assign(regionInfos);
+    } catch (InterruptedException e) {
+      LOG.error("Caught " + e + " during round-robin assignment");
+      InterruptedIOException ie = new InterruptedIOException(e.getMessage());
+      ie.initCause(e);
+      throw ie;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
index 351f364..053a946 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
@@ -109,6 +109,9 @@ public class TestAdmin {
 
   @After
   public void tearDown() throws Exception {
+    for (HTableDescriptor htd : this.admin.listTables()) {
+      TEST_UTIL.deleteTable(htd.getName());
+    }
   }
 
   @Test (timeout=300000)
@@ -341,6 +344,48 @@ public class TestAdmin {
   }
 
   @Test (timeout=300000)
+  public void testTruncateTable() throws IOException {
+    testTruncateTable(TableName.valueOf("testTruncateTable"), false);
+  }
+
+  @Test (timeout=300000)
+  public void testTruncateTablePreservingSplits() throws IOException {
+    testTruncateTable(TableName.valueOf("testTruncateTablePreservingSplits"), true);
+  }
+
+  private void testTruncateTable(final TableName tableName, boolean preserveSplits)
+      throws IOException {
+    byte[][] splitKeys = new byte[2][];
+    splitKeys[0] = Bytes.toBytes(4);
+    splitKeys[1] = Bytes.toBytes(8);
+
+    // Create & Fill the table
+    HTable table = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY, splitKeys);
+    try {
+      TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10);
+      assertEquals(10, TEST_UTIL.countRows(table));
+    } finally {
+      table.close();
+    }
+    assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+
+    // Truncate & Verify
+    this.admin.disableTable(tableName);
+    this.admin.truncateTable(tableName, preserveSplits);
+    table = new HTable(TEST_UTIL.getConfiguration(), tableName);
+    try {
+      assertEquals(0, TEST_UTIL.countRows(table));
+    } finally {
+      table.close();
+    }
+    if (preserveSplits) {
+      assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+    } else {
+      assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+    }
+  }
+
+  @Test (timeout=300000)
   public void testGetTableDescriptor() throws IOException {
     HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
     HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
@@ -366,7 +411,7 @@ public class TestAdmin {
            assertTrue(exceptionThrown);
        }
    }
-  
+
   /**
    * Verify schema modification takes.
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 9e9bd80..a241c29 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -71,6 +71,8 @@ public class TestMasterObserver {
     private boolean postCreateTableCalled;
     private boolean preDeleteTableCalled;
     private boolean postDeleteTableCalled;
+    private boolean preTruncateTableCalled;
+    private boolean postTruncateTableCalled;
     private boolean preModifyTableCalled;
     private boolean postModifyTableCalled;
     private boolean preCreateNamespaceCalled;
@@ -119,6 +121,8 @@ public class TestMasterObserver {
     private boolean postCreateTableHandlerCalled;
     private boolean preDeleteTableHandlerCalled;
     private boolean postDeleteTableHandlerCalled;
+    private boolean preTruncateTableHandlerCalled;
+    private boolean postTruncateTableHandlerCalled;
     private boolean preAddColumnHandlerCalled;
     private boolean postAddColumnHandlerCalled;
     private boolean preModifyColumnHandlerCalled;
@@ -143,6 +147,8 @@ public class TestMasterObserver {
       postCreateTableCalled = false;
       preDeleteTableCalled = false;
       postDeleteTableCalled = false;
+      preTruncateTableCalled = false;
+      postTruncateTableCalled = false;
       preModifyTableCalled = false;
       postModifyTableCalled = false;
       preCreateNamespaceCalled = false;
@@ -185,6 +191,8 @@ public class TestMasterObserver {
       postCreateTableHandlerCalled = false;
       preDeleteTableHandlerCalled = false;
       postDeleteTableHandlerCalled = false;
+      preTruncateTableHandlerCalled = false;
+      postTruncateTableHandlerCalled = false;
       preModifyTableHandlerCalled = false;
       postModifyTableHandlerCalled = false;
       preAddColumnHandlerCalled = false;
@@ -250,6 +258,29 @@ public class TestMasterObserver {
     }
 
     @Override
+    public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> env,
+        TableName tableName) throws IOException {
+      if (bypass) {
+        env.bypass();
+      }
+      preTruncateTableCalled = true;
+    }
+
+    @Override
+    public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> env,
+        TableName tableName) throws IOException {
+      postTruncateTableCalled = true;
+    }
+
+    public boolean wasTruncateTableCalled() {
+      return preTruncateTableCalled && postTruncateTableCalled;
+    }
+
+    public boolean preTruncateTableCalledOnly() {
+      return preTruncateTableCalled && !postTruncateTableCalled;
+    }
+
+    @Override
     public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
         TableName tableName, HTableDescriptor htd) throws IOException {
       if (bypass) {
@@ -768,6 +799,32 @@ public class TestMasterObserver {
     public boolean wasDeleteTableHandlerCalledOnly() {
       return preDeleteTableHandlerCalled && !postDeleteTableHandlerCalled;
     }
+
+    @Override
+    public void preTruncateTableHandler(
+        ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
+        throws IOException {
+      if (bypass) {
+        env.bypass();
+      }
+      preTruncateTableHandlerCalled = true;
+    }
+
+    @Override
+    public void postTruncateTableHandler(
+        ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+        throws IOException {
+      postTruncateTableHandlerCalled = true;
+    }
+
+    public boolean wasTruncateTableHandlerCalled() {
+      return preTruncateTableHandlerCalled && postTruncateTableHandlerCalled;
+    }
+
+    public boolean wasTruncateTableHandlerCalledOnly() {
+      return preTruncateTableHandlerCalled && !postTruncateTableHandlerCalled;
+    }
+
     @Override
     public void preModifyTableHandler(
         ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName,
@@ -1050,7 +1107,12 @@ public class TestMasterObserver {
     assertTrue("Second column family should be modified",
       cp.preModifyColumnCalledOnly());
 
+    // truncate table
+    admin.truncateTable(TEST_TABLE, false);
+
     // delete table
+    admin.disableTable(TEST_TABLE);
+    assertTrue(admin.isTableDisabled(TEST_TABLE));
     admin.deleteTable(TEST_TABLE);
     assertFalse("Test table should have been deleted",
         admin.tableExists(TEST_TABLE));
@@ -1280,7 +1342,7 @@ public class TestMasterObserver {
     try {
       UTIL.createMultiRegions(table, TEST_FAMILY);
       UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
-  
+
       NavigableMap<HRegionInfo, ServerName> regions = table.getRegionLocations();
       Map.Entry<HRegionInfo, ServerName> firstGoodPair = null;
       for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
@@ -1312,15 +1374,15 @@ public class TestMasterObserver {
         firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
       assertTrue("Coprocessor should have been called on region move",
         cp.wasMoveCalled());
-  
+
       // make sure balancer is on
       master.balanceSwitch(true);
       assertTrue("Coprocessor should have been called on balance switch",
           cp.wasBalanceSwitchCalled());
-  
+
       // turn balancer off
       master.balanceSwitch(false);
-  
+
       // wait for assignments to finish, if any
       AssignmentManager mgr = master.getAssignmentManager();
       Collection<RegionState> transRegions =
@@ -1328,7 +1390,7 @@ public class TestMasterObserver {
       for (RegionState state : transRegions) {
         mgr.getRegionStates().waitOnRegionToClearRegionsInTransition(state.getRegion());
       }
-  
+
       // move half the open regions from RS 0 to RS 1
       HRegionServer rs = cluster.getRegionServer(0);
       byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 9037bb3..ade7635 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
@@ -343,6 +343,10 @@ public class TestCatalogJanitor {
     public void deleteTable(TableName tableName) throws IOException { }
 
     @Override
+    public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { }
+
+
+    @Override
     public void modifyTable(TableName tableName, HTableDescriptor descriptor)
         throws IOException { }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index e9e6b1a..714c340 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -320,6 +321,22 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test
+  public void testTableTruncate() throws Exception {
+    AccessTestAction truncateTable = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER
+            .preTruncateTable(ObserverContext.createAndPrepare(CP_ENV, null),
+              TEST_TABLE.getTableName());
+        return null;
+      }
+    };
+
+    verifyAllowed(truncateTable, SUPERUSER, USER_ADMIN);
+    verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE,USER_CREATE, USER_OWNER);
+  }
+
+  @Test
   public void testAddColumn() throws Exception {
     final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
     AccessTestAction action = new AccessTestAction() {
@@ -2259,4 +2276,21 @@ public class TestAccessController extends SecureTestUtil {
     TEST_UTIL.getMiniHBaseCluster().getMaster().deleteNamespace(namespace);
   }
 
+  @Test
+  public void testTruncatePerms() throws Exception {
+    try {
+      List<UserPermission> existingPerms = AccessControlClient.getUserPermissions(conf, TEST_TABLE
+          .getTableName().getNameAsString());
+      assertTrue(existingPerms != null);
+      assertTrue(existingPerms.size() > 1);
+      TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE.getTableName());
+      TEST_UTIL.getHBaseAdmin().truncateTable(TEST_TABLE.getTableName(), true);
+      List<UserPermission> perms = AccessControlClient.getUserPermissions(conf, TEST_TABLE
+          .getTableName().getNameAsString());
+      assertTrue(perms != null);
+      assertEquals(existingPerms.size(), perms.size());
+    } catch (Throwable e) {
+      throw new HBaseIOException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 428d390..1b545c3 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -340,11 +340,24 @@ module Hbase
       yield 'Disabling table...' if block_given?
       @admin.disableTable(table_name)
 
-      yield 'Dropping table...' if block_given?
-      @admin.deleteTable(table_name)
-
-      yield 'Creating table...' if block_given?
-      @admin.createTable(table_description)
+      begin
+        yield 'Truncating table...' if block_given?
+        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false)
+      rescue => e
+        # Handle the compatibility case, where the truncate method doesn't exists on the Master
+        raise e unless e.respond_to?(:cause) && e.cause != nil
+        rootCause = e.cause
+        if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
+          # Handle the compatibility case, where the truncate method doesn't exists on the Master
+          yield 'Dropping table...' if block_given?
+          @admin.deleteTable(table_name)
+
+          yield 'Creating table...' if block_given?
+          @admin.createTable(table_description)
+        else
+          raise e
+        end
+      end
     end
 
     #----------------------------------------------------------------------------------------------
@@ -357,11 +370,24 @@ module Hbase
       yield 'Disabling table...' if block_given?
       disable(table_name)
 
-      yield 'Dropping table...' if block_given?
-      drop(table_name)
-
-      yield 'Creating table with region boundaries...' if block_given?
-      @admin.createTable(table_description, splits)
+      begin
+        yield 'Truncating table...' if block_given?
+        @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true)
+      rescue => e
+        # Handle the compatibility case, where the truncate method doesn't exists on the Master
+        raise e unless e.respond_to?(:cause) && e.cause != nil
+        rootCause = e.cause
+        if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
+          # Handle the compatibility case, where the truncate method doesn't exists on the Master
+          yield 'Dropping table...' if block_given?
+          @admin.deleteTable(table_name)
+
+          yield 'Creating table with region boundaries...' if block_given?
+          @admin.createTable(table_description, splits)
+        else
+          raise e
+        end
+      end
     end
 
     #----------------------------------------------------------------------------------------------


[2/2] git commit: HBASE-12142 Truncate command does not preserve ACLs table (Vandana Ayyalasomayajula)

Posted by ap...@apache.org.
HBASE-12142 Truncate command does not preserve ACLs table (Vandana Ayyalasomayajula)


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

Branch: refs/heads/0.98
Commit: 5b1380c3c691007625d624096f57559124518288
Parents: 714e836
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Oct 24 16:33:01 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Oct 24 16:33:01 2014 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   21 +
 .../hadoop/hbase/client/HConnectionManager.java |    8 +
 .../hadoop/hbase/protobuf/RequestConverter.java |   16 +
 .../hbase/protobuf/generated/MasterProtos.java  | 1248 ++++++++++++++++--
 hbase-protocol/src/main/protobuf/Master.proto   |   12 +
 .../hbase/coprocessor/BaseMasterObserver.java   |   22 +
 .../hbase/coprocessor/MasterObserver.java       |   46 +
 .../org/apache/hadoop/hbase/master/HMaster.java |   30 +
 .../hbase/master/MasterCoprocessorHost.java     |   68 +
 .../hadoop/hbase/master/MasterServices.java     |    8 +
 .../master/handler/TruncateTableHandler.java    |  247 ++++
 .../hbase/security/access/AccessController.java |   34 +
 .../visibility/VisibilityController.java        |   20 +
 .../hadoop/hbase/util/ModifyRegionUtils.java    |   21 +
 .../apache/hadoop/hbase/client/TestAdmin.java   |   47 +-
 .../hbase/coprocessor/TestMasterObserver.java   |   72 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |    4 +
 .../security/access/TestAccessController.java   |   34 +
 hbase-shell/src/main/ruby/hbase/admin.rb        |   46 +-
 19 files changed, 1910 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 3cf57e5..022c382 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
@@ -128,6 +128,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
 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.UnassignRegionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
@@ -3403,4 +3404,24 @@ public class HBaseAdmin implements Abortable, Closeable {
     return new RegionServerCoprocessorRpcChannel(connection, sn);
   }
 
+  /**
+   * 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
+   */
+  public void truncateTable(final TableName tableName, final boolean preserveSplits)
+      throws IOException {
+    executeCallable(new MasterCallable<Void>(getConnection()) {
+      @Override
+      public Void call() throws ServiceException {
+        LOG.info("Started truncate of " + tableName);
+        TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
+        tableName, preserveSplits);
+        master.truncateTable(null, req);
+        return null;
+      }
+    });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
index 30046b2..4fa5a8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
@@ -110,6 +110,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunnin
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
+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.UnassignRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
@@ -2128,6 +2130,12 @@ public class HConnectionManager {
             throws ServiceException {
           return stub.getClusterStatus(controller, request);
         }
+
+        @Override
+        public TruncateTableResponse truncateTable(RpcController controller,
+            TruncateTableRequest request) throws ServiceException {
+          return stub.truncateTable(controller, request);
+        }
       };
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 756bc43..f5c128b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import org.apache.hadoop.hbase.util.ByteStringer;
@@ -1142,6 +1143,21 @@ public final class RequestConverter {
   }
 
   /**
+   * Creates a protocol buffer TruncateTableRequest
+   *
+   * @param tableName name of table to truncate
+   * @param preserveSplits True if the splits should be preserved
+   * @return a TruncateTableRequest
+   */
+  public static TruncateTableRequest buildTruncateTableRequest(final TableName tableName,
+        boolean preserveSplits) {
+    TruncateTableRequest.Builder builder = TruncateTableRequest.newBuilder();
+    builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
+    builder.setPreserveSplits(preserveSplits);
+    return builder.build();
+  }
+
+  /**
    * Creates a protocol buffer EnableTableRequest
    *
    * @param tableName

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 83a2152..e52ad17 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
@@ -39552,6 +39552,995 @@ public final class MasterProtos {
     // @@protoc_insertion_point(class_scope:IsProcedureDoneResponse)
   }
 
+  public interface TruncateTableRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required .TableName tableName = 1;
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    // optional bool preserveSplits = 2 [default = false];
+    /**
+     * <code>optional bool preserveSplits = 2 [default = false];</code>
+     */
+    boolean hasPreserveSplits();
+    /**
+     * <code>optional bool preserveSplits = 2 [default = false];</code>
+     */
+    boolean getPreserveSplits();
+  }
+  /**
+   * Protobuf type {@code TruncateTableRequest}
+   */
+  public static final class TruncateTableRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements TruncateTableRequestOrBuilder {
+    // Use TruncateTableRequest.newBuilder() to construct.
+    private TruncateTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TruncateTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TruncateTableRequest defaultInstance;
+    public static TruncateTableRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TruncateTableRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TruncateTableRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              preserveSplits_ = input.readBool();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TruncateTableRequest> PARSER =
+        new com.google.protobuf.AbstractParser<TruncateTableRequest>() {
+      public TruncateTableRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TruncateTableRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TruncateTableRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required .TableName tableName = 1;
+    public static final int TABLENAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_;
+    }
+    /**
+     * <code>required .TableName tableName = 1;</code>
+     */
+    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_;
+    }
+
+    // optional bool preserveSplits = 2 [default = false];
+    public static final int PRESERVESPLITS_FIELD_NUMBER = 2;
+    private boolean preserveSplits_;
+    /**
+     * <code>optional bool preserveSplits = 2 [default = false];</code>
+     */
+    public boolean hasPreserveSplits() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bool preserveSplits = 2 [default = false];</code>
+     */
+    public boolean getPreserveSplits() {
+      return preserveSplits_;
+    }
+
+    private void initFields() {
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      preserveSplits_ = false;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!getTableName().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, tableName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, preserveSplits_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, tableName_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(2, preserveSplits_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasPreserveSplits() == other.hasPreserveSplits());
+      if (hasPreserveSplits()) {
+        result = result && (getPreserveSplits()
+            == other.getPreserveSplits());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLENAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasPreserveSplits()) {
+        hash = (37 * hash) + PRESERVESPLITS_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getPreserveSplits());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code TruncateTableRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        preserveSplits_ = false;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.preserveSplits_ = preserveSplits_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (other.hasPreserveSplits()) {
+          setPreserveSplits(other.getPreserveSplits());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!getTableName().isInitialized()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required .TableName tableName = 1;
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_;
+        }
+      }
+      /**
+       * <code>required .TableName tableName = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  tableName_,
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      // optional bool preserveSplits = 2 [default = false];
+      private boolean preserveSplits_ ;
+      /**
+       * <code>optional bool preserveSplits = 2 [default = false];</code>
+       */
+      public boolean hasPreserveSplits() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bool preserveSplits = 2 [default = false];</code>
+       */
+      public boolean getPreserveSplits() {
+        return preserveSplits_;
+      }
+      /**
+       * <code>optional bool preserveSplits = 2 [default = false];</code>
+       */
+      public Builder setPreserveSplits(boolean value) {
+        bitField0_ |= 0x00000002;
+        preserveSplits_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool preserveSplits = 2 [default = false];</code>
+       */
+      public Builder clearPreserveSplits() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        preserveSplits_ = false;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:TruncateTableRequest)
+    }
+
+    static {
+      defaultInstance = new TruncateTableRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:TruncateTableRequest)
+  }
+
+  public interface TruncateTableResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code TruncateTableResponse}
+   */
+  public static final class TruncateTableResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements TruncateTableResponseOrBuilder {
+    // Use TruncateTableResponse.newBuilder() to construct.
+    private TruncateTableResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TruncateTableResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TruncateTableResponse defaultInstance;
+    public static TruncateTableResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TruncateTableResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TruncateTableResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TruncateTableResponse> PARSER =
+        new com.google.protobuf.AbstractParser<TruncateTableResponse>() {
+      public TruncateTableResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TruncateTableResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TruncateTableResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) obj;
+
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code TruncateTableResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse build() {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      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);
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      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;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:TruncateTableResponse)
+    }
+
+    static {
+      defaultInstance = new TruncateTableResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:TruncateTableResponse)
+  }
+
   /**
    * Protobuf service {@code MasterService}
    */
@@ -40077,6 +41066,18 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
           com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse> done);
 
+      /**
+       * <code>rpc truncateTable(.TruncateTableRequest) returns (.TruncateTableResponse);</code>
+       *
+       * <pre>
+       ** Truncate a table 
+       * </pre>
+       */
+      public abstract void truncateTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done);
+
     }
 
     public static com.google.protobuf.Service newReflectiveService(
@@ -40410,6 +41411,14 @@ public final class MasterProtos {
           impl.listTableNamesByNamespace(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void truncateTable(
+            com.google.protobuf.RpcController controller,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done) {
+          impl.truncateTable(controller, request, done);
+        }
+
       };
     }
 
@@ -40514,6 +41523,8 @@ public final class MasterProtos {
               return impl.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request);
             case 40:
               return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request);
+            case 41:
+              return impl.truncateTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -40610,6 +41621,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance();
             case 40:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance();
+            case 41:
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -40706,6 +41719,8 @@ public final class MasterProtos {
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance();
             case 40:
               return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance();
+            case 41:
+              return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -41231,6 +42246,18 @@ public final class MasterProtos {
         org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
         com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse> done);
 
+    /**
+     * <code>rpc truncateTable(.TruncateTableRequest) returns (.TruncateTableResponse);</code>
+     *
+     * <pre>
+     ** Truncate a table 
+     * </pre>
+     */
+    public abstract void truncateTable(
+        com.google.protobuf.RpcController controller,
+        org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done);
+
     public static final
         com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -41458,6 +42485,11 @@ public final class MasterProtos {
             com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse>specializeCallback(
               done));
           return;
+        case 41:
+          this.truncateTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -41554,6 +42586,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance();
         case 40:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance();
+        case 41:
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -41650,6 +42684,8 @@ public final class MasterProtos {
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance();
         case 40:
           return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance();
+        case 41:
+          return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -42285,6 +43321,21 @@ public final class MasterProtos {
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.class,
             org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance()));
       }
+
+      public  void truncateTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(41),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class,
+            org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -42497,6 +43548,11 @@ public final class MasterProtos {
           com.google.protobuf.RpcController controller,
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request)
           throws com.google.protobuf.ServiceException;
+
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request)
+          throws com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -42997,9 +44053,19 @@ public final class MasterProtos {
           org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance());
       }
 
-    }
 
-    // @@protoc_insertion_point(class_scope:MasterService)
+      public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(41),
+          controller,
+          request,
+          org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance());
+      }
+
+    }
   }
 
   private static com.google.protobuf.Descriptors.Descriptor
@@ -43402,6 +44468,16 @@ public final class MasterProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_IsProcedureDoneResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_TruncateTableRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_TruncateTableRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_TruncateTableResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_TruncateTableResponse_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -43516,82 +44592,86 @@ public final class MasterProtos {
       "oneRequest\022(\n\tprocedure\030\001 \001(\0132\025.Procedur" +
       "eDescription\"W\n\027IsProcedureDoneResponse\022" +
       "\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snapshot\030\002 \001(\0132\025" +
-      ".ProcedureDescription2\356\026\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\0228\n\013EnableTable\022\023",
-      ".EnableTableRequest\032\024.EnableTableRespons" +
-      "e\022;\n\014DisableTable\022\024.DisableTableRequest\032" +
-      "\025.DisableTableResponse\0228\n\013ModifyTable\022\023." +
-      "ModifyTableRequest\032\024.ModifyTableResponse" +
-      "\0228\n\013CreateTable\022\023.CreateTableRequest\032\024.C" +
-      "reateTableResponse\022/\n\010Shutdown\022\020.Shutdow" +
-      "nRequest\032\021.ShutdownResponse\0225\n\nStopMaste" +
-      "r\022\022.StopMasterRequest\032\023.StopMasterRespon" +
-      "se\022,\n\007Balance\022\017.BalanceRequest\032\020.Balance" +
-      "Response\022M\n\022SetBalancerRunning\022\032.SetBala",
-      "ncerRunningRequest\032\033.SetBalancerRunningR" +
-      "esponse\022A\n\016RunCatalogScan\022\026.RunCatalogSc" +
-      "anRequest\032\027.RunCatalogScanResponse\022S\n\024En" +
-      "ableCatalogJanitor\022\034.EnableCatalogJanito" +
-      "rRequest\032\035.EnableCatalogJanitorResponse\022" +
-      "\\\n\027IsCatalogJanitorEnabled\022\037.IsCatalogJa" +
-      "nitorEnabledRequest\032 .IsCatalogJanitorEn" +
-      "abledResponse\022L\n\021ExecMasterService\022\032.Cop" +
-      "rocessorServiceRequest\032\033.CoprocessorServ" +
-      "iceResponse\022/\n\010Snapshot\022\020.SnapshotReques",
-      "t\032\021.SnapshotResponse\022V\n\025GetCompletedSnap" +
-      "shots\022\035.GetCompletedSnapshotsRequest\032\036.G" +
-      "etCompletedSnapshotsResponse\022A\n\016DeleteSn" +
-      "apshot\022\026.DeleteSnapshotRequest\032\027.DeleteS" +
-      "napshotResponse\022A\n\016IsSnapshotDone\022\026.IsSn" +
-      "apshotDoneRequest\032\027.IsSnapshotDoneRespon" +
-      "se\022D\n\017RestoreSnapshot\022\027.RestoreSnapshotR" +
-      "equest\032\030.RestoreSnapshotResponse\022V\n\025IsRe" +
-      "storeSnapshotDone\022\035.IsRestoreSnapshotDon" +
-      "eRequest\032\036.IsRestoreSnapshotDoneResponse",
-      "\022>\n\rExecProcedure\022\025.ExecProcedureRequest" +
-      "\032\026.ExecProcedureResponse\022D\n\017IsProcedureD" +
-      "one\022\027.IsProcedureDoneRequest\032\030.IsProcedu" +
-      "reDoneResponse\022D\n\017ModifyNamespace\022\027.Modi" +
-      "fyNamespaceRequest\032\030.ModifyNamespaceResp" +
-      "onse\022D\n\017CreateNamespace\022\027.CreateNamespac" +
-      "eRequest\032\030.CreateNamespaceResponse\022D\n\017De" +
-      "leteNamespace\022\027.DeleteNamespaceRequest\032\030" +
-      ".DeleteNamespaceResponse\022Y\n\026GetNamespace" +
-      "Descriptor\022\036.GetNamespaceDescriptorReque",
-      "st\032\037.GetNamespaceDescriptorResponse\022_\n\030L" +
-      "istNamespaceDescriptors\022 .ListNamespaceD" +
-      "escriptorsRequest\032!.ListNamespaceDescrip" +
-      "torsResponse\022t\n\037ListTableDescriptorsByNa" +
-      "mespace\022\'.ListTableDescriptorsByNamespac" +
-      "eRequest\032(.ListTableDescriptorsByNamespa" +
-      "ceResponse\022b\n\031ListTableNamesByNamespace\022" +
-      "!.ListTableNamesByNamespaceRequest\032\".Lis" +
-      "tTableNamesByNamespaceResponseBB\n*org.ap" +
-      "ache.hadoop.hbase.protobuf.generatedB\014Ma",
-      "sterProtosH\001\210\001\001\240\001\001"
+      ".ProcedureDescription\"T\n\024TruncateTableRe" +
+      "quest\022\035\n\ttableName\030\001 \002(\0132\n.TableName\022\035\n\016" +
+      "preserveSplits\030\002 \001(\010:\005false\"\027\n\025TruncateT" +
+      "ableResponse2\256\027\n\rMasterService\022S\n\024GetSch" +
+      "emaAlterStatus\022\034.GetSchemaAlterStatusReq",
+      "uest\032\035.GetSchemaAlterStatusResponse\022P\n\023G" +
+      "etTableDescriptors\022\033.GetTableDescriptors" +
+      "Request\032\034.GetTableDescriptorsResponse\022>\n" +
+      "\rGetTableNames\022\025.GetTableNamesRequest\032\026." +
+      "GetTableNamesResponse\022G\n\020GetClusterStatu" +
+      "s\022\030.GetClusterStatusRequest\032\031.GetCluster" +
+      "StatusResponse\022D\n\017IsMasterRunning\022\027.IsMa" +
+      "sterRunningRequest\032\030.IsMasterRunningResp" +
+      "onse\0222\n\tAddColumn\022\021.AddColumnRequest\032\022.A" +
+      "ddColumnResponse\022;\n\014DeleteColumn\022\024.Delet",
+      "eColumnRequest\032\025.DeleteColumnResponse\022;\n" +
+      "\014ModifyColumn\022\024.ModifyColumnRequest\032\025.Mo" +
+      "difyColumnResponse\0225\n\nMoveRegion\022\022.MoveR" +
+      "egionRequest\032\023.MoveRegionResponse\022Y\n\026Dis" +
+      "patchMergingRegions\022\036.DispatchMergingReg" +
+      "ionsRequest\032\037.DispatchMergingRegionsResp" +
+      "onse\022;\n\014AssignRegion\022\024.AssignRegionReque" +
+      "st\032\025.AssignRegionResponse\022A\n\016UnassignReg" +
+      "ion\022\026.UnassignRegionRequest\032\027.UnassignRe" +
+      "gionResponse\022>\n\rOfflineRegion\022\025.OfflineR",
+      "egionRequest\032\026.OfflineRegionResponse\0228\n\013" +
+      "DeleteTable\022\023.DeleteTableRequest\032\024.Delet" +
+      "eTableResponse\0228\n\013EnableTable\022\023.EnableTa" +
+      "bleRequest\032\024.EnableTableResponse\022;\n\014Disa" +
+      "bleTable\022\024.DisableTableRequest\032\025.Disable" +
+      "TableResponse\0228\n\013ModifyTable\022\023.ModifyTab" +
+      "leRequest\032\024.ModifyTableResponse\0228\n\013Creat" +
+      "eTable\022\023.CreateTableRequest\032\024.CreateTabl" +
+      "eResponse\022/\n\010Shutdown\022\020.ShutdownRequest\032" +
+      "\021.ShutdownResponse\0225\n\nStopMaster\022\022.StopM",
+      "asterRequest\032\023.StopMasterResponse\022,\n\007Bal" +
+      "ance\022\017.BalanceRequest\032\020.BalanceResponse\022" +
+      "M\n\022SetBalancerRunning\022\032.SetBalancerRunni" +
+      "ngRequest\032\033.SetBalancerRunningResponse\022A" +
+      "\n\016RunCatalogScan\022\026.RunCatalogScanRequest" +
+      "\032\027.RunCatalogScanResponse\022S\n\024EnableCatal" +
+      "ogJanitor\022\034.EnableCatalogJanitorRequest\032" +
+      "\035.EnableCatalogJanitorResponse\022\\\n\027IsCata" +
+      "logJanitorEnabled\022\037.IsCatalogJanitorEnab" +
+      "ledRequest\032 .IsCatalogJanitorEnabledResp",
+      "onse\022L\n\021ExecMasterService\022\032.CoprocessorS" +
+      "erviceRequest\032\033.CoprocessorServiceRespon" +
+      "se\022/\n\010Snapshot\022\020.SnapshotRequest\032\021.Snaps" +
+      "hotResponse\022V\n\025GetCompletedSnapshots\022\035.G" +
+      "etCompletedSnapshotsRequest\032\036.GetComplet" +
+      "edSnapshotsResponse\022A\n\016DeleteSnapshot\022\026." +
+      "DeleteSnapshotRequest\032\027.DeleteSnapshotRe" +
+      "sponse\022A\n\016IsSnapshotDone\022\026.IsSnapshotDon" +
+      "eRequest\032\027.IsSnapshotDoneResponse\022D\n\017Res" +
+      "toreSnapshot\022\027.RestoreSnapshotRequest\032\030.",
+      "RestoreSnapshotResponse\022V\n\025IsRestoreSnap" +
+      "shotDone\022\035.IsRestoreSnapshotDoneRequest\032" +
+      "\036.IsRestoreSnapshotDoneResponse\022>\n\rExecP" +
+      "rocedure\022\025.ExecProcedureRequest\032\026.ExecPr" +
+      "ocedureResponse\022D\n\017IsProcedureDone\022\027.IsP" +
+      "rocedureDoneRequest\032\030.IsProcedureDoneRes" +
+      "ponse\022D\n\017ModifyNamespace\022\027.ModifyNamespa" +
+      "ceRequest\032\030.ModifyNamespaceResponse\022D\n\017C" +
+      "reateNamespace\022\027.CreateNamespaceRequest\032" +
+      "\030.CreateNamespaceResponse\022D\n\017DeleteNames",
+      "pace\022\027.DeleteNamespaceRequest\032\030.DeleteNa" +
+      "mespaceResponse\022Y\n\026GetNamespaceDescripto" +
+      "r\022\036.GetNamespaceDescriptorRequest\032\037.GetN" +
+      "amespaceDescriptorResponse\022_\n\030ListNamesp" +
+      "aceDescriptors\022 .ListNamespaceDescriptor" +
+      "sRequest\032!.ListNamespaceDescriptorsRespo" +
+      "nse\022t\n\037ListTableDescriptorsByNamespace\022\'" +
+      ".ListTableDescriptorsByNamespaceRequest\032" +
+      "(.ListTableDescriptorsByNamespaceRespons" +
+      "e\022b\n\031ListTableNamesByNamespace\022!.ListTab",
+      "leNamesByNamespaceRequest\032\".ListTableNam" +
+      "esByNamespaceResponse\022>\n\rtruncateTable\022\025" +
+      ".TruncateTableRequest\032\026.TruncateTableRes" +
+      "ponseBB\n*org.apache.hadoop.hbase.protobu" +
+      "f.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -44078,6 +45158,18 @@ public final class MasterProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_IsProcedureDoneResponse_descriptor,
               new java.lang.String[] { "Done", "Snapshot", });
+          internal_static_TruncateTableRequest_descriptor =
+            getDescriptor().getMessageTypes().get(80);
+          internal_static_TruncateTableRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_TruncateTableRequest_descriptor,
+              new java.lang.String[] { "TableName", "PreserveSplits", });
+          internal_static_TruncateTableResponse_descriptor =
+            getDescriptor().getMessageTypes().get(81);
+          internal_static_TruncateTableResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_TruncateTableResponse_descriptor,
+              new java.lang.String[] { });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 b1e1a72..bdd053c 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -349,6 +349,14 @@ message IsProcedureDoneResponse {
         optional ProcedureDescription snapshot = 2;
 }
 
+message TruncateTableRequest {
+          required TableName tableName = 1;
+            optional bool preserveSplits = 2 [default = false];
+}
+
+message TruncateTableResponse {
+}
+
 service MasterService {
   /** Used by the client to get the number of regions that have received the updated schema */
   rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -547,4 +555,8 @@ service MasterService {
   /** returns a list of tables for a given namespace*/
   rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
     returns(ListTableNamesByNamespaceResponse);
+
+  /** Truncate a table */
+    rpc truncateTable(TruncateTableRequest)
+        returns(TruncateTableResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
index c85f3c6..29be365 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
@@ -83,6 +83,28 @@ public class BaseMasterObserver implements MasterObserver {
   }
 
   @Override
+  public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
+
+  @Override
+  public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException {
+  }
+
+  @Override
+  public void preTruncateTableHandler(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+      throws IOException {
+  }
+
+  @Override
+  public void postTruncateTableHandler(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+      throws IOException {
+  }
+
+  @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
       TableName tableName, HTableDescriptor htd) throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index bd0d845..d6cb609 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -134,6 +134,52 @@ public interface MasterObserver extends Coprocessor {
       final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException;
 
+
+  /**
+   * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+   * table.  Called as part of truncate table RPC call.
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   */
+  void preTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException;
+
+  /**
+   * Called after the truncateTable operation has been requested.  Called as part
+   * of truncate table RPC call.
+   * The truncate is synchronous, so this method will be called when the
+   * truncate operation is terminated.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   */
+  void postTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      TableName tableName) throws IOException;
+
+  /**
+   * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+   * table.  Called as part of truncate table handler and it is sync
+   * to the truncate RPC call.
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   */
+  void preTruncateTableHandler(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+      throws IOException;
+
+  /**
+   * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+   * table.  Called as part of truncate table handler and it is sync to the
+   * truncate RPC call.
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param tableName the name of the table
+   */
+  void postTruncateTableHandler(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+      throws IOException;
+
   /**
    * Called prior to modifying a table's properties.  Called as part of modify
    * table RPC call.

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 af5e297..4e55cb4 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
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
+import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -191,6 +192,8 @@ 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.StopMasterResponse;
+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.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
@@ -3225,4 +3228,31 @@ MasterServices, Server {
     }
   }
 
+  @Override
+  public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
+    checkInitialized();
+    if (cpHost != null) {
+      cpHost.preTruncateTable(tableName);
+    }
+    LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
+    TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
+    handler.prepare();
+    handler.process();
+    if (cpHost != null) {
+      cpHost.postTruncateTable(tableName);
+    }
+  }
+
+
+  @Override
+  public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
+      throws ServiceException {
+    try {
+      truncateTable(ProtobufUtil.toTableName(request.getTableName()), request.getPreserveSplits());
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+    return TruncateTableResponse.newBuilder().build();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 60669c4..34a8b0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -225,6 +225,74 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preTruncateTable(TableName tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env: coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver)env.getInstance()).preTruncateTable(ctx, tableName);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+  }
+
+  public void postTruncateTable(TableName tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env: coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver)env.getInstance()).postTruncateTable(ctx, tableName);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+  }
+
+  public void preTruncateTableHandler(TableName tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env : coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver) env.getInstance()).preTruncateTableHandler(ctx, tableName);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+  }
+
+  public void postTruncateTableHandler(TableName tableName) throws IOException {
+    ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+    for (MasterEnvironment env : coprocessors) {
+      if (env.getInstance() instanceof MasterObserver) {
+        ctx = ObserverContext.createAndPrepare(env, ctx);
+        try {
+          ((MasterObserver) env.getInstance()).postTruncateTableHandler(ctx, tableName);
+        } catch (Throwable e) {
+          handleCoprocessorThrowable(env, e);
+        }
+        if (ctx.shouldComplete()) {
+          break;
+        }
+      }
+    }
+  }
+
   public void preModifyTable(final TableName tableName, final HTableDescriptor htd)
       throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/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 a5765ae..c402758 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
@@ -98,6 +98,14 @@ public interface MasterServices extends Server {
   void deleteTable(final TableName tableName) throws IOException;
 
   /**
+   * Truncate a table
+   * @param tableName The table name
+   * @param preserveSplits True if the splits should be preserved
+   * @throws IOException
+   */
+  public void truncateTable(final TableName tableName, boolean preserveSplits) throws IOException;
+
+  /**
    * Modify the descriptor of an existing table
    * @param tableName The table name
    * @param descriptor The updated table descriptor